github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/disttae/logtailreplay/partition.go (about)

     1  // Copyright 2022 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package logtailreplay
    16  
    17  import (
    18  	"bytes"
    19  	"context"
    20  	"sync"
    21  	"sync/atomic"
    22  
    23  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    24  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/checkpoint"
    25  
    26  	"github.com/matrixorigin/matrixone/pkg/container/types"
    27  	"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
    28  )
    29  
    30  // a partition corresponds to a dn
    31  type Partition struct {
    32  	//lock is used to protect pointer of PartitionState from concurrent mutation
    33  	lock  chan struct{}
    34  	state atomic.Pointer[PartitionState]
    35  
    36  	// assuming checkpoints will be consumed once
    37  	checkpointConsumed atomic.Bool
    38  
    39  	//current partitionState can serve snapshot read only if start <= ts <= end
    40  	mu struct {
    41  		sync.Mutex
    42  		start types.TS
    43  		end   types.TS
    44  	}
    45  
    46  	TableInfo   TableInfo
    47  	TableInfoOK bool
    48  }
    49  
    50  type TableInfo struct {
    51  	ID            uint64
    52  	Name          string
    53  	PrimarySeqnum int
    54  }
    55  
    56  func (p *Partition) CanServe(ts types.TS) bool {
    57  	p.mu.Lock()
    58  	defer p.mu.Unlock()
    59  	return ts.GreaterEq(&p.mu.start) && ts.LessEq(&p.mu.end)
    60  }
    61  
    62  func NewPartition() *Partition {
    63  	lock := make(chan struct{}, 1)
    64  	lock <- struct{}{}
    65  	ret := &Partition{
    66  		lock: lock,
    67  	}
    68  	ret.mu.start = types.MaxTs()
    69  	ret.state.Store(NewPartitionState(false))
    70  	return ret
    71  }
    72  
    73  type RowID types.Rowid
    74  
    75  func (r RowID) Less(than RowID) bool {
    76  	return bytes.Compare(r[:], than[:]) < 0
    77  }
    78  
    79  func (p *Partition) Snapshot() *PartitionState {
    80  	return p.state.Load()
    81  }
    82  
    83  func (*Partition) CheckPoint(ctx context.Context, ts timestamp.Timestamp) error {
    84  	panic("unimplemented")
    85  }
    86  
    87  func (p *Partition) MutateState() (*PartitionState, func()) {
    88  	curState := p.state.Load()
    89  	state := curState.Copy()
    90  	return state, func() {
    91  		if !p.state.CompareAndSwap(curState, state) {
    92  			panic("concurrent mutation")
    93  		}
    94  	}
    95  }
    96  
    97  func (p *Partition) Lock(ctx context.Context) error {
    98  	select {
    99  	case <-p.lock:
   100  		return nil
   101  	case <-ctx.Done():
   102  		return ctx.Err()
   103  	}
   104  }
   105  
   106  func (p *Partition) Unlock() {
   107  	p.lock <- struct{}{}
   108  }
   109  
   110  func (p *Partition) IsValid() bool {
   111  	p.mu.Lock()
   112  	defer p.mu.Unlock()
   113  	return p.mu.start.LessEq(&p.mu.end)
   114  }
   115  
   116  func (p *Partition) IsEmpty() bool {
   117  	p.mu.Lock()
   118  	defer p.mu.Unlock()
   119  	return p.mu.start == types.MaxTs()
   120  }
   121  
   122  func (p *Partition) UpdateStart(ts types.TS) {
   123  	p.mu.Lock()
   124  	defer p.mu.Unlock()
   125  	if p.mu.start != types.MaxTs() {
   126  		p.mu.start = ts
   127  	}
   128  }
   129  
   130  // [start, end]
   131  func (p *Partition) UpdateDuration(start types.TS, end types.TS) {
   132  	p.mu.Lock()
   133  	defer p.mu.Unlock()
   134  	p.mu.start = start
   135  	p.mu.end = end
   136  }
   137  
   138  func (p *Partition) GetDuration() (types.TS, types.TS) {
   139  	p.mu.Lock()
   140  	defer p.mu.Unlock()
   141  	return p.mu.start, p.mu.end
   142  }
   143  
   144  func (p *Partition) ConsumeSnapCkps(
   145  	_ context.Context,
   146  	ckps []*checkpoint.CheckpointEntry,
   147  	fn func(
   148  		ckp *checkpoint.CheckpointEntry,
   149  		state *PartitionState,
   150  	) error,
   151  ) (
   152  	err error,
   153  ) {
   154  	if len(ckps) == 0 {
   155  		return nil
   156  	}
   157  	//Notice that checkpoints must contain only one or zero global checkpoint
   158  	//followed by zero or multi continuous incremental checkpoints.
   159  	state := p.state.Load()
   160  	start := types.MaxTs()
   161  	end := types.TS{}
   162  	for _, ckp := range ckps {
   163  		if err = fn(ckp, state); err != nil {
   164  			return
   165  		}
   166  		if ckp.GetType() == checkpoint.ET_Global {
   167  			start = ckp.GetEnd()
   168  		}
   169  		if ckp.GetType() == checkpoint.ET_Incremental {
   170  			ckpstart := ckp.GetStart()
   171  			if ckpstart.Less(&start) {
   172  				start = ckpstart
   173  			}
   174  			ckpend := ckp.GetEnd()
   175  			if ckpend.Greater(&end) {
   176  				end = ckpend
   177  			}
   178  		}
   179  	}
   180  	if end.IsEmpty() {
   181  		//only one global checkpoint.
   182  		end = start
   183  	}
   184  	p.UpdateDuration(start, end)
   185  	if !p.IsValid() {
   186  		return moerr.NewInternalErrorNoCtx("invalid checkpoints duration")
   187  	}
   188  	return nil
   189  }
   190  
   191  func (p *Partition) ConsumeCheckpoints(
   192  	ctx context.Context,
   193  	fn func(
   194  		checkpoint string,
   195  		state *PartitionState,
   196  	) error,
   197  ) (
   198  	err error,
   199  ) {
   200  
   201  	if p.checkpointConsumed.Load() {
   202  		return nil
   203  	}
   204  	curState := p.state.Load()
   205  	if len(curState.checkpoints) == 0 {
   206  		p.UpdateDuration(types.TS{}, types.MaxTs())
   207  		return nil
   208  	}
   209  
   210  	lockErr := p.Lock(ctx)
   211  	if lockErr != nil {
   212  		return lockErr
   213  	}
   214  	defer p.Unlock()
   215  
   216  	curState = p.state.Load()
   217  	if len(curState.checkpoints) == 0 {
   218  		p.UpdateDuration(types.TS{}, types.MaxTs())
   219  		return nil
   220  	}
   221  
   222  	state := curState.Copy()
   223  
   224  	if err := state.consumeCheckpoints(fn); err != nil {
   225  		return err
   226  	}
   227  
   228  	p.UpdateDuration(state.start, types.MaxTs())
   229  
   230  	if !p.state.CompareAndSwap(curState, state) {
   231  		panic("concurrent mutation")
   232  	}
   233  
   234  	p.checkpointConsumed.Store(true)
   235  
   236  	return
   237  }
   238  
   239  func (p *Partition) Truncate(ctx context.Context, ids [2]uint64, ts types.TS) error {
   240  	err := p.Lock(ctx)
   241  	if err != nil {
   242  		return err
   243  	}
   244  	defer p.Unlock()
   245  	curState := p.state.Load()
   246  
   247  	state := curState.Copy()
   248  
   249  	state.truncate(ids, ts)
   250  
   251  	//TODO::update partition's start and end
   252  
   253  	if !p.state.CompareAndSwap(curState, state) {
   254  		panic("concurrent mutation")
   255  	}
   256  
   257  	return nil
   258  }