github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/db/open.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 db
    16  
    17  import (
    18  	"bytes"
    19  	"context"
    20  	"path"
    21  	"sync/atomic"
    22  	"time"
    23  
    24  	"github.com/BurntSushi/toml"
    25  	"github.com/matrixorigin/matrixone/pkg/container/types"
    26  	"github.com/matrixorigin/matrixone/pkg/logutil"
    27  	"github.com/matrixorigin/matrixone/pkg/objectio"
    28  	v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2"
    29  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog"
    30  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    31  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers"
    32  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/checkpoint"
    33  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/dbutils"
    34  	gc2 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/gc"
    35  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/merge"
    36  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/gc"
    37  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail"
    38  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/model"
    39  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options"
    40  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables"
    41  	w "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks/worker"
    42  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase"
    43  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnimpl"
    44  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal"
    45  )
    46  
    47  const (
    48  	WALDir = "wal"
    49  )
    50  
    51  func fillRuntimeOptions(opts *options.Options) {
    52  	common.RuntimeCNMergeMemControl.Store(opts.MergeCfg.CNMergeMemControlHint)
    53  	common.RuntimeMinCNMergeSize.Store(opts.MergeCfg.CNTakeOverExceed)
    54  	common.RuntimeCNTakeOverAll.Store(opts.MergeCfg.CNTakeOverAll)
    55  	common.RuntimeOverallFlushMemCap.Store(opts.CheckpointCfg.OverallFlushMemControl)
    56  	if opts.IsStandalone {
    57  		common.IsStandaloneBoost.Store(true)
    58  	}
    59  	if opts.MergeCfg.CNStandaloneTake {
    60  		common.ShouldStandaloneCNTakeOver.Store(true)
    61  	}
    62  }
    63  
    64  func Open(ctx context.Context, dirname string, opts *options.Options) (db *DB, err error) {
    65  	dbLocker, err := createDBLock(dirname)
    66  
    67  	logutil.Info("open-tae", common.OperationField("Start"),
    68  		common.OperandField("open"))
    69  	totalTime := time.Now()
    70  
    71  	if err != nil {
    72  		return nil, err
    73  	}
    74  	defer func() {
    75  		if dbLocker != nil {
    76  			dbLocker.Close()
    77  		}
    78  		logutil.Info("open-tae", common.OperationField("End"),
    79  			common.OperandField("open"),
    80  			common.AnyField("cost", time.Since(totalTime)),
    81  			common.AnyField("err", err))
    82  	}()
    83  
    84  	opts = opts.FillDefaults(dirname)
    85  	fillRuntimeOptions(opts)
    86  
    87  	wbuf := &bytes.Buffer{}
    88  	werr := toml.NewEncoder(wbuf).Encode(opts)
    89  	logutil.Info("open-tae", common.OperationField("Config"),
    90  		common.AnyField("toml", wbuf.String()), common.ErrorField(werr))
    91  	serviceDir := path.Join(dirname, "data")
    92  	if opts.Fs == nil {
    93  		// TODO:fileservice needs to be passed in as a parameter
    94  		opts.Fs = objectio.TmpNewFileservice(ctx, path.Join(dirname, "data"))
    95  	}
    96  
    97  	db = &DB{
    98  		Dir:          dirname,
    99  		Opts:         opts,
   100  		Closed:       new(atomic.Value),
   101  		usageMemo:    logtail.NewTNUsageMemo(),
   102  		CNMergeSched: merge.NewTaskServiceGetter(opts.TaskServiceGetter),
   103  	}
   104  	fs := objectio.NewObjectFS(opts.Fs, serviceDir)
   105  	transferTable := model.NewTransferTable[*model.TransferHashPage](db.Opts.TransferTableTTL)
   106  
   107  	switch opts.LogStoreT {
   108  	case options.LogstoreBatchStore:
   109  		db.Wal = wal.NewDriverWithBatchStore(opts.Ctx, dirname, WALDir, nil)
   110  	case options.LogstoreLogservice:
   111  		db.Wal = wal.NewDriverWithLogservice(opts.Ctx, opts.Lc)
   112  	}
   113  	scheduler := newTaskScheduler(db, db.Opts.SchedulerCfg.AsyncWorkers, db.Opts.SchedulerCfg.IOWorkers)
   114  	db.Runtime = dbutils.NewRuntime(
   115  		dbutils.WithRuntimeTransferTable(transferTable),
   116  		dbutils.WithRuntimeObjectFS(fs),
   117  		dbutils.WithRuntimeSmallPool(dbutils.MakeDefaultSmallPool("small-vector-pool")),
   118  		dbutils.WithRuntimeTransientPool(dbutils.MakeDefaultTransientPool("trasient-vector-pool")),
   119  		dbutils.WithRuntimeScheduler(scheduler),
   120  		dbutils.WithRuntimeOptions(db.Opts),
   121  	)
   122  
   123  	dataFactory := tables.NewDataFactory(
   124  		db.Runtime, db.Dir,
   125  	)
   126  	if db.Catalog, err = catalog.OpenCatalog(db.usageMemo); err != nil {
   127  		return
   128  	}
   129  
   130  	// Init and start txn manager
   131  	txnStoreFactory := txnimpl.TxnStoreFactory(
   132  		opts.Ctx,
   133  		db.Catalog,
   134  		db.Wal,
   135  		db.Runtime,
   136  		dataFactory,
   137  		opts.MaxMessageSize,
   138  	)
   139  	txnFactory := txnimpl.TxnFactory(db.Catalog)
   140  	db.TxnMgr = txnbase.NewTxnManager(txnStoreFactory, txnFactory, db.Opts.Clock)
   141  	db.LogtailMgr = logtail.NewManager(
   142  		db.Runtime,
   143  		int(db.Opts.LogtailCfg.PageSize),
   144  		db.TxnMgr.Now,
   145  	)
   146  	db.Runtime.Now = db.TxnMgr.Now
   147  	db.TxnMgr.CommitListener.AddTxnCommitListener(db.LogtailMgr)
   148  	db.TxnMgr.Start(opts.Ctx)
   149  	db.LogtailMgr.Start()
   150  	db.BGCheckpointRunner = checkpoint.NewRunner(
   151  		opts.Ctx,
   152  		db.Runtime,
   153  		db.Catalog,
   154  		logtail.NewDirtyCollector(db.LogtailMgr, db.Opts.Clock, db.Catalog, new(catalog.LoopProcessor)),
   155  		db.Wal,
   156  		checkpoint.WithFlushInterval(opts.CheckpointCfg.FlushInterval),
   157  		checkpoint.WithCollectInterval(opts.CheckpointCfg.ScanInterval),
   158  		checkpoint.WithMinCount(int(opts.CheckpointCfg.MinCount)),
   159  		checkpoint.WithCheckpointBlockRows(opts.CheckpointCfg.BlockRows),
   160  		checkpoint.WithCheckpointSize(opts.CheckpointCfg.Size),
   161  		checkpoint.WithMinIncrementalInterval(opts.CheckpointCfg.IncrementalInterval),
   162  		checkpoint.WithGlobalMinCount(int(opts.CheckpointCfg.GlobalMinCount)),
   163  		checkpoint.WithGlobalVersionInterval(opts.CheckpointCfg.GlobalVersionInterval),
   164  		checkpoint.WithReserveWALEntryCount(opts.CheckpointCfg.ReservedWALEntryCount))
   165  
   166  	now := time.Now()
   167  	checkpointed, ckpLSN, valid, err := db.BGCheckpointRunner.Replay(dataFactory)
   168  	if err != nil {
   169  		panic(err)
   170  	}
   171  	logutil.Info("open-tae", common.OperationField("replay"),
   172  		common.OperandField("checkpoints"),
   173  		common.AnyField("cost", time.Since(now)),
   174  		common.AnyField("checkpointed", checkpointed.ToString()))
   175  
   176  	now = time.Now()
   177  	db.Replay(dataFactory, checkpointed, ckpLSN, valid)
   178  	db.Catalog.ReplayTableRows()
   179  
   180  	// checkObjectState(db)
   181  	logutil.Info("open-tae", common.OperationField("replay"),
   182  		common.OperandField("wal"),
   183  		common.AnyField("cost", time.Since(now)))
   184  
   185  	db.DBLocker, dbLocker = dbLocker, nil
   186  
   187  	// Init timed scanner
   188  	scanner := NewDBScanner(db, nil)
   189  	db.MergeHandle = newMergeTaskBuilder(db)
   190  	scanner.RegisterOp(db.MergeHandle)
   191  	db.Wal.Start()
   192  	db.BGCheckpointRunner.Start()
   193  
   194  	db.BGScanner = w.NewHeartBeater(
   195  		opts.CheckpointCfg.ScanInterval,
   196  		scanner)
   197  	db.BGScanner.Start()
   198  	// TODO: WithGCInterval requires configuration parameters
   199  	cleaner := gc2.NewCheckpointCleaner(opts.Ctx, fs, db.BGCheckpointRunner, opts.GCCfg.DisableGC)
   200  	cleaner.AddChecker(
   201  		func(item any) bool {
   202  			checkpoint := item.(*checkpoint.CheckpointEntry)
   203  			ts := types.BuildTS(time.Now().UTC().UnixNano()-int64(opts.GCCfg.GCTTL), 0)
   204  			endTS := checkpoint.GetEnd()
   205  			return !endTS.GreaterEq(&ts)
   206  		})
   207  	db.DiskCleaner = gc2.NewDiskCleaner(cleaner)
   208  	db.DiskCleaner.Start()
   209  	// Init gc manager at last
   210  	// TODO: clean-try-gc requires configuration parameters
   211  	db.GCManager = gc.NewManager(
   212  		gc.WithCronJob(
   213  			"clean-transfer-table",
   214  			opts.CheckpointCfg.FlushInterval,
   215  			func(_ context.Context) (err error) {
   216  				db.Runtime.PrintVectorPoolUsage()
   217  				db.Runtime.TransferDelsMap.Prune(opts.TransferTableTTL)
   218  				transferTable.RunTTL(time.Now())
   219  				return
   220  			}),
   221  
   222  		gc.WithCronJob(
   223  			"disk-gc",
   224  			opts.GCCfg.ScanGCInterval,
   225  			func(ctx context.Context) (err error) {
   226  				db.DiskCleaner.GC(ctx)
   227  				return
   228  			}),
   229  		gc.WithCronJob(
   230  			"checkpoint-gc",
   231  			opts.CheckpointCfg.GCCheckpointInterval,
   232  			func(ctx context.Context) error {
   233  				if opts.CheckpointCfg.DisableGCCheckpoint {
   234  					return nil
   235  				}
   236  				consumed := db.DiskCleaner.GetCleaner().GetMaxConsumed()
   237  				if consumed == nil {
   238  					return nil
   239  				}
   240  				return db.BGCheckpointRunner.GCByTS(ctx, consumed.GetEnd())
   241  			}),
   242  		gc.WithCronJob(
   243  			"catalog-gc",
   244  			opts.CatalogCfg.GCInterval,
   245  			func(ctx context.Context) error {
   246  				if opts.CatalogCfg.DisableGC {
   247  					return nil
   248  				}
   249  				consumed := db.DiskCleaner.GetCleaner().GetMaxConsumed()
   250  				if consumed == nil {
   251  					return nil
   252  				}
   253  				db.Catalog.GCByTS(ctx, consumed.GetEnd())
   254  				return nil
   255  			}),
   256  		gc.WithCronJob(
   257  			"logtail-gc",
   258  			opts.CheckpointCfg.GCCheckpointInterval,
   259  			func(ctx context.Context) error {
   260  				logutil.Info(db.Runtime.ExportLogtailStats())
   261  				ckp := db.BGCheckpointRunner.MaxCheckpoint()
   262  				if ckp != nil {
   263  					// use previous end to gc logtail
   264  					ts := types.BuildTS(ckp.GetStart().Physical(), 0) // GetStart is previous + 1, reset it here
   265  					db.LogtailMgr.GCByTS(ctx, ts)
   266  				}
   267  				return nil
   268  			},
   269  		),
   270  	)
   271  
   272  	db.GCManager.Start()
   273  
   274  	go TaeMetricsTask(ctx)
   275  
   276  	// For debug or test
   277  	// logutil.Info(db.Catalog.SimplePPString(common.PPL2))
   278  	return
   279  }
   280  
   281  // TODO: remove it
   282  // func checkObjectState(db *DB) {
   283  // 	p := &catalog.LoopProcessor{}
   284  // 	p.ObjectFn = func(oe *catalog.ObjectEntry) error {
   285  // 		if oe.IsAppendable() == oe.IsSorted() {
   286  // 			panic(fmt.Sprintf("logic err %v", oe.ID.String()))
   287  // 		}
   288  // 		return nil
   289  // 	}
   290  // 	db.Catalog.RecurLoop(p)
   291  // }
   292  
   293  func TaeMetricsTask(ctx context.Context) {
   294  	logutil.Info("tae metrics task started")
   295  	defer logutil.Info("tae metrics task exit")
   296  
   297  	timer := time.NewTicker(time.Second * 10)
   298  	for {
   299  		select {
   300  		case <-ctx.Done():
   301  			return
   302  		case <-timer.C:
   303  			mpoolAllocatorSubTask()
   304  		}
   305  	}
   306  
   307  }
   308  
   309  func mpoolAllocatorSubTask() {
   310  	v2.MemTAEDefaultAllocatorGauge.Set(float64(common.DefaultAllocator.CurrNB()))
   311  	v2.MemTAEDefaultHighWaterMarkGauge.Set(float64(common.DefaultAllocator.Stats().HighWaterMark.Load()))
   312  
   313  	v2.MemTAEMutableAllocatorGauge.Set(float64(common.MutMemAllocator.CurrNB()))
   314  	v2.MemTAEMutableHighWaterMarkGauge.Set(float64(common.MutMemAllocator.Stats().HighWaterMark.Load()))
   315  
   316  	v2.MemTAESmallAllocatorGauge.Set(float64(common.SmallAllocator.CurrNB()))
   317  	v2.MemTAESmallHighWaterMarkGauge.Set(float64(common.SmallAllocator.Stats().HighWaterMark.Load()))
   318  
   319  	v2.MemTAEVectorPoolDefaultAllocatorGauge.Set(float64(containers.GetDefaultVectorPoolALLocator().CurrNB()))
   320  	v2.MemTAEVectorPoolDefaultHighWaterMarkGauge.Set(float64(containers.GetDefaultVectorPoolALLocator().Stats().HighWaterMark.Load()))
   321  
   322  	v2.MemTAELogtailAllocatorGauge.Set(float64(common.LogtailAllocator.CurrNB()))
   323  	v2.MemTAELogtailHighWaterMarkGauge.Set(float64(common.LogtailAllocator.Stats().HighWaterMark.Load()))
   324  
   325  	v2.MemTAECheckpointAllocatorGauge.Set(float64(common.CheckpointAllocator.CurrNB()))
   326  	v2.MemTAECheckpointHighWaterMarkGauge.Set(float64(common.CheckpointAllocator.Stats().HighWaterMark.Load()))
   327  
   328  	v2.MemTAEMergeAllocatorGauge.Set(float64(common.MergeAllocator.CurrNB()))
   329  	v2.MemTAEMergeHighWaterMarkGauge.Set(float64(common.MergeAllocator.Stats().HighWaterMark.Load()))
   330  
   331  	v2.MemTAEWorkSpaceAllocatorGauge.Set(float64(common.WorkspaceAllocator.CurrNB()))
   332  	v2.MemTAEWorkSpaceHighWaterMarkGauge.Set(float64(common.WorkspaceAllocator.Stats().HighWaterMark.Load()))
   333  
   334  	v2.MemTAEDebugAllocatorGauge.Set(float64(common.DebugAllocator.CurrNB()))
   335  	v2.MemTAEDebugHighWaterMarkGauge.Set(float64(common.DebugAllocator.Stats().HighWaterMark.Load()))
   336  
   337  }