github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logstore/store/checkpoint.go (about)

     1  // Copyright 2021 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 store
    16  
    17  import (
    18  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    19  	driverEntry "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/driver/entry"
    20  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/entry"
    21  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm"
    22  )
    23  
    24  func (w *StoreImpl) RangeCheckpoint(gid uint32, start, end uint64) (ckpEntry entry.Entry, err error) {
    25  	ckpEntry = w.makeRangeCheckpointEntry(gid, start, end)
    26  	drentry, _, err := w.doAppend(GroupCKP, ckpEntry)
    27  	if err == sm.ErrClose {
    28  		return nil, err
    29  	}
    30  	if err != nil {
    31  		panic(err)
    32  	}
    33  	_, err = w.checkpointQueue.Enqueue(drentry)
    34  	if err != nil {
    35  		panic(err)
    36  	}
    37  	return
    38  }
    39  
    40  func (w *StoreImpl) makeRangeCheckpointEntry(gid uint32, start, end uint64) (ckpEntry entry.Entry) {
    41  	info := &entry.Info{
    42  		Group: entry.GTCKp,
    43  		Checkpoints: []*entry.CkpRanges{{
    44  			Group:  gid,
    45  			Ranges: common.NewClosedIntervalsByInterval(&common.ClosedInterval{Start: start, End: end}),
    46  		}},
    47  	}
    48  	ckpEntry = entry.GetBase()
    49  	ckpEntry.SetType(entry.IOET_WALEntry_Checkpoint)
    50  	ckpEntry.SetInfo(info)
    51  	return
    52  }
    53  
    54  func (w *StoreImpl) onLogCKPInfoQueue(items ...any) {
    55  	for _, item := range items {
    56  		e := item.(*driverEntry.Entry)
    57  		err := e.WaitDone()
    58  		if err != nil {
    59  			panic(err)
    60  		}
    61  		w.logCheckpointInfo(e.Info)
    62  	}
    63  	w.onCheckpoint()
    64  }
    65  
    66  func (w *StoreImpl) onCheckpoint() {
    67  	w.StoreInfo.onCheckpoint()
    68  	w.ckpCkp()
    69  }
    70  
    71  func (w *StoreImpl) ckpCkp() {
    72  	e := w.makeInternalCheckpointEntry()
    73  	driverEntry, _, err := w.doAppend(GroupInternal, e)
    74  	if err == sm.ErrClose {
    75  		return
    76  	}
    77  	if err != nil {
    78  		panic(err)
    79  	}
    80  	w.truncatingQueue.Enqueue(driverEntry)
    81  	err = e.WaitDone()
    82  	if err != nil {
    83  		panic(err)
    84  	}
    85  	e.Free()
    86  }
    87  
    88  func (w *StoreImpl) onTruncatingQueue(items ...any) {
    89  	for _, item := range items {
    90  		e := item.(*driverEntry.Entry)
    91  		err := e.WaitDone()
    92  		if err != nil {
    93  			panic(err)
    94  		}
    95  		w.logCheckpointInfo(e.Info)
    96  	}
    97  	gid, driverLsn := w.getDriverCheckpointed()
    98  	if gid == 0 {
    99  		return
   100  	}
   101  	w.driverCheckpointing.Store(driverLsn)
   102  	_, err := w.truncateQueue.Enqueue(struct{}{})
   103  	if err != nil {
   104  		panic(err)
   105  	}
   106  }
   107  
   108  func (w *StoreImpl) onTruncateQueue(items ...any) {
   109  	lsn := w.driverCheckpointing.Load()
   110  	if lsn != w.driverCheckpointed {
   111  		err := w.driver.Truncate(lsn)
   112  		for err != nil {
   113  			lsn = w.driverCheckpointing.Load()
   114  			err = w.driver.Truncate(lsn)
   115  		}
   116  		w.gcWalDriverLsnMap(lsn)
   117  		w.driverCheckpointed = lsn
   118  	}
   119  }