github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/db/checkpoint/testutils.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 checkpoint
    16  
    17  import (
    18  	"context"
    19  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/dbutils"
    20  	"time"
    21  
    22  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    23  	"github.com/matrixorigin/matrixone/pkg/container/types"
    24  	"github.com/matrixorigin/matrixone/pkg/logutil"
    25  	"github.com/matrixorigin/matrixone/pkg/util/fault"
    26  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    27  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail"
    28  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal"
    29  	"go.uber.org/zap"
    30  )
    31  
    32  type TestRunner interface {
    33  	EnableCheckpoint()
    34  	DisableCheckpoint()
    35  
    36  	CleanPenddingCheckpoint()
    37  	ForceGlobalCheckpoint(end types.TS, versionInterval time.Duration) error
    38  	ForceGlobalCheckpointSynchronously(ctx context.Context, end types.TS, versionInterval time.Duration) error
    39  	ForceCheckpointForBackup(end types.TS) (string, error)
    40  	ForceIncrementalCheckpoint(end types.TS, truncate bool) error
    41  	IsAllChangesFlushed(start, end types.TS, printTree bool) bool
    42  	MaxLSNInRange(end types.TS) uint64
    43  
    44  	ExistPendingEntryToGC() bool
    45  	MaxGlobalCheckpoint() *CheckpointEntry
    46  	MaxCheckpoint() *CheckpointEntry
    47  	ForceFlush(ts types.TS, ctx context.Context, duration time.Duration) (err error)
    48  	ForceFlushWithInterval(ts types.TS, ctx context.Context, forceDuration, flushInterval time.Duration) (err error)
    49  	GetDirtyCollector() logtail.Collector
    50  }
    51  
    52  // DisableCheckpoint stops generating checkpoint
    53  func (r *runner) DisableCheckpoint() {
    54  	r.disabled.Store(true)
    55  }
    56  
    57  func (r *runner) EnableCheckpoint() {
    58  	r.disabled.Store(false)
    59  }
    60  
    61  func (r *runner) CleanPenddingCheckpoint() {
    62  	prev := r.MaxCheckpoint()
    63  	if prev == nil {
    64  		return
    65  	}
    66  	if !prev.IsFinished() {
    67  		r.storage.Lock()
    68  		r.storage.entries.Delete(prev)
    69  		r.storage.Unlock()
    70  	}
    71  	if prev.IsRunning() {
    72  		logutil.Warnf("Delete a running checkpoint entry")
    73  	}
    74  	prev = r.MaxGlobalCheckpoint()
    75  	if prev == nil {
    76  		return
    77  	}
    78  	if !prev.IsFinished() {
    79  		r.storage.Lock()
    80  		r.storage.entries.Delete(prev)
    81  		r.storage.Unlock()
    82  	}
    83  	if prev.IsRunning() {
    84  		logutil.Warnf("Delete a running checkpoint entry")
    85  	}
    86  }
    87  
    88  func (r *runner) ForceGlobalCheckpoint(end types.TS, versionInterval time.Duration) error {
    89  	if versionInterval == 0 {
    90  		versionInterval = r.options.globalVersionInterval
    91  	}
    92  	if r.GetPenddingIncrementalCount() != 0 {
    93  		end = r.MaxCheckpoint().GetEnd()
    94  		r.globalCheckpointQueue.Enqueue(&globalCheckpointContext{
    95  			force:    true,
    96  			end:      end,
    97  			interval: versionInterval,
    98  		})
    99  		return nil
   100  	}
   101  	timeout := time.After(versionInterval)
   102  	for {
   103  		select {
   104  		case <-timeout:
   105  			return moerr.NewInternalError(r.ctx, "timeout")
   106  		default:
   107  			err := r.ForceIncrementalCheckpoint(end, false)
   108  			if err != nil {
   109  				if dbutils.IsRetrieableCheckpoint(err) {
   110  					interval := versionInterval.Milliseconds() / 400
   111  					time.Sleep(time.Duration(interval))
   112  					break
   113  				}
   114  				return err
   115  			}
   116  			r.globalCheckpointQueue.Enqueue(&globalCheckpointContext{
   117  				force:    true,
   118  				end:      end,
   119  				interval: versionInterval,
   120  			})
   121  			return nil
   122  		}
   123  	}
   124  }
   125  
   126  func (r *runner) ForceGlobalCheckpointSynchronously(ctx context.Context, end types.TS, versionInterval time.Duration) error {
   127  	prevGlobalEnd := types.TS{}
   128  	global, _ := r.storage.globals.Max()
   129  	if global != nil {
   130  		prevGlobalEnd = global.end
   131  	}
   132  
   133  	r.ForceGlobalCheckpoint(end, versionInterval)
   134  
   135  	op := func() (ok bool, err error) {
   136  		global, _ := r.storage.globals.Max()
   137  		if global == nil {
   138  			return false, nil
   139  		}
   140  		return global.end.Greater(&prevGlobalEnd), nil
   141  	}
   142  	err := common.RetryWithIntervalAndTimeout(
   143  		op,
   144  		time.Minute,
   145  		r.options.forceFlushCheckInterval, false)
   146  	if err != nil {
   147  		return moerr.NewInternalError(ctx, "force global checkpoint failed: %v", err)
   148  	}
   149  	return nil
   150  }
   151  
   152  func (r *runner) ForceFlushWithInterval(ts types.TS, ctx context.Context, forceDuration, flushInterval time.Duration) (err error) {
   153  	makeCtx := func() *DirtyCtx {
   154  		tree := r.source.ScanInRangePruned(types.TS{}, ts)
   155  		tree.GetTree().Compact()
   156  		if tree.IsEmpty() {
   157  			return nil
   158  		}
   159  		entry := logtail.NewDirtyTreeEntry(types.TS{}, ts, tree.GetTree())
   160  		dirtyCtx := new(DirtyCtx)
   161  		dirtyCtx.tree = entry
   162  		dirtyCtx.force = true
   163  		// logutil.Infof("try flush %v",tree.String())
   164  		return dirtyCtx
   165  	}
   166  	op := func() (ok bool, err error) {
   167  		dirtyCtx := makeCtx()
   168  		if dirtyCtx == nil {
   169  			return true, nil
   170  		}
   171  		if _, err = r.dirtyEntryQueue.Enqueue(dirtyCtx); err != nil {
   172  			return true, nil
   173  		}
   174  		return false, nil
   175  	}
   176  
   177  	if forceDuration == 0 {
   178  		forceDuration = r.options.forceFlushTimeout
   179  	}
   180  	err = common.RetryWithIntervalAndTimeout(
   181  		op,
   182  		forceDuration,
   183  		flushInterval, false)
   184  	if err != nil {
   185  		return moerr.NewInternalError(ctx, "force flush failed: %v", err)
   186  	}
   187  	_, sarg, _ := fault.TriggerFault("tae: flush timeout")
   188  	if sarg != "" {
   189  		err = moerr.NewInternalError(ctx, sarg)
   190  	}
   191  	return
   192  
   193  }
   194  func (r *runner) ForceFlush(ts types.TS, ctx context.Context, forceDuration time.Duration) (err error) {
   195  	return r.ForceFlushWithInterval(ts, ctx, forceDuration, r.options.forceFlushCheckInterval)
   196  }
   197  
   198  func (r *runner) ForceIncrementalCheckpoint(end types.TS, truncate bool) error {
   199  	now := time.Now()
   200  	prev := r.MaxCheckpoint()
   201  	if prev != nil && !prev.IsFinished() {
   202  		return moerr.NewPrevCheckpointNotFinished()
   203  	}
   204  
   205  	if prev != nil && end.LessEq(&prev.end) {
   206  		return nil
   207  	}
   208  	var (
   209  		err      error
   210  		errPhase string
   211  		start    types.TS
   212  		fatal    bool
   213  		fields   []zap.Field
   214  	)
   215  
   216  	if prev != nil {
   217  		start = prev.end.Next()
   218  	}
   219  
   220  	entry := NewCheckpointEntry(start, end, ET_Incremental)
   221  	logutil.Info(
   222  		"Checkpoint-Start-Force",
   223  		zap.String("entry", entry.String()),
   224  	)
   225  
   226  	defer func() {
   227  		if err != nil {
   228  			logger := logutil.Error
   229  			if fatal {
   230  				logger = logutil.Fatal
   231  			}
   232  			logger(
   233  				"Checkpoint-Error-Force",
   234  				zap.String("entry", entry.String()),
   235  				zap.String("phase", errPhase),
   236  				zap.Error(err),
   237  				zap.Duration("cost", time.Since(now)),
   238  			)
   239  		} else {
   240  			fields = append(fields, zap.Duration("cost", time.Since(now)))
   241  			fields = append(fields, zap.String("entry", entry.String()))
   242  			logutil.Info(
   243  				"Checkpoint-End-Force",
   244  				fields...,
   245  			)
   246  		}
   247  	}()
   248  
   249  	r.storage.Lock()
   250  	r.storage.entries.Set(entry)
   251  	r.storage.Unlock()
   252  
   253  	if fields, err = r.doIncrementalCheckpoint(entry); err != nil {
   254  		errPhase = "do-ckp"
   255  		return err
   256  	}
   257  
   258  	var lsn, lsnToTruncate uint64
   259  	if truncate {
   260  		lsn = r.source.GetMaxLSN(entry.start, entry.end)
   261  		if lsn > r.options.reservedWALEntryCount {
   262  			lsnToTruncate = lsn - r.options.reservedWALEntryCount
   263  		}
   264  		entry.ckpLSN = lsn
   265  		entry.truncateLSN = lsnToTruncate
   266  	}
   267  
   268  	if err = r.saveCheckpoint(
   269  		entry.start, entry.end, lsn, lsnToTruncate,
   270  	); err != nil {
   271  		errPhase = "save-ckp"
   272  		return err
   273  	}
   274  	entry.SetState(ST_Finished)
   275  
   276  	if truncate {
   277  		var e wal.LogEntry
   278  		if e, err = r.wal.RangeCheckpoint(1, lsnToTruncate); err != nil {
   279  			errPhase = "wal-ckp"
   280  			fatal = true
   281  			return err
   282  		}
   283  		if err = e.WaitDone(); err != nil {
   284  			errPhase = "wait-wal-ckp"
   285  			fatal = true
   286  			return err
   287  		}
   288  	}
   289  	return nil
   290  }
   291  
   292  func (r *runner) ForceCheckpointForBackup(end types.TS) (location string, err error) {
   293  	prev := r.MaxCheckpoint()
   294  	if prev != nil && !prev.IsFinished() {
   295  		return "", moerr.NewInternalError(r.ctx, "prev checkpoint not finished")
   296  	}
   297  	start := types.TS{}
   298  	if prev != nil {
   299  		start = prev.end.Next()
   300  	}
   301  	entry := NewCheckpointEntry(start, end, ET_Incremental)
   302  	r.storage.Lock()
   303  	r.storage.entries.Set(entry)
   304  	now := time.Now()
   305  	r.storage.Unlock()
   306  	if _, err = r.doIncrementalCheckpoint(entry); err != nil {
   307  		return
   308  	}
   309  	var lsn, lsnToTruncate uint64
   310  	lsn = r.source.GetMaxLSN(entry.start, entry.end)
   311  	if lsn > r.options.reservedWALEntryCount {
   312  		lsnToTruncate = lsn - r.options.reservedWALEntryCount
   313  	}
   314  	entry.ckpLSN = lsn
   315  	entry.truncateLSN = lsnToTruncate
   316  	if err = r.saveCheckpoint(entry.start, entry.end, lsn, lsnToTruncate); err != nil {
   317  		return
   318  	}
   319  	backupTime := time.Now().UTC()
   320  	currTs := types.BuildTS(backupTime.UnixNano(), 0)
   321  	backup := NewCheckpointEntry(end.Next(), currTs, ET_Incremental)
   322  	location, err = r.doCheckpointForBackup(backup)
   323  	if err != nil {
   324  		return
   325  	}
   326  	entry.SetState(ST_Finished)
   327  	e, err := r.wal.RangeCheckpoint(1, lsnToTruncate)
   328  	if err != nil {
   329  		panic(err)
   330  	}
   331  	if err = e.WaitDone(); err != nil {
   332  		panic(err)
   333  	}
   334  	logutil.Infof("checkpoint for backup %s, takes %s", entry.String(), time.Since(now))
   335  	return location, nil
   336  }
   337  
   338  func (r *runner) IsAllChangesFlushed(start, end types.TS, printTree bool) bool {
   339  	tree := r.source.ScanInRangePruned(start, end)
   340  	tree.GetTree().Compact()
   341  	if printTree && !tree.IsEmpty() {
   342  		logutil.Infof("%v", tree.String())
   343  	}
   344  	return tree.IsEmpty()
   345  }