github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logtail/utils.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 logtail
    16  
    17  import (
    18  	"context"
    19  	"fmt"
    20  	"sort"
    21  	"time"
    22  
    23  	"github.com/matrixorigin/matrixone/pkg/fileservice"
    24  	"go.uber.org/zap"
    25  
    26  	pkgcatalog "github.com/matrixorigin/matrixone/pkg/catalog"
    27  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    28  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    29  	"github.com/matrixorigin/matrixone/pkg/container/batch"
    30  	"github.com/matrixorigin/matrixone/pkg/container/types"
    31  	"github.com/matrixorigin/matrixone/pkg/container/vector"
    32  	"github.com/matrixorigin/matrixone/pkg/logutil"
    33  	"github.com/matrixorigin/matrixone/pkg/objectio"
    34  	"github.com/matrixorigin/matrixone/pkg/pb/api"
    35  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio"
    36  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog"
    37  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    38  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers"
    39  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/data"
    40  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnimpl"
    41  )
    42  
    43  const DefaultCheckpointBlockRows = 10000
    44  const DefaultCheckpointSize = 512 * 1024 * 1024
    45  
    46  const (
    47  	CheckpointVersion1  uint32 = 1
    48  	CheckpointVersion2  uint32 = 2
    49  	CheckpointVersion3  uint32 = 3
    50  	CheckpointVersion4  uint32 = 4
    51  	CheckpointVersion5  uint32 = 5
    52  	CheckpointVersion6  uint32 = 6
    53  	CheckpointVersion7  uint32 = 7
    54  	CheckpointVersion8  uint32 = 8
    55  	CheckpointVersion9  uint32 = 9
    56  	CheckpointVersion10 uint32 = 10
    57  	CheckpointVersion11 uint32 = 11
    58  
    59  	CheckpointCurrentVersion = CheckpointVersion11
    60  )
    61  
    62  const (
    63  	MetaIDX uint16 = iota
    64  
    65  	DBInsertIDX
    66  	DBInsertTxnIDX
    67  	DBDeleteIDX
    68  	DBDeleteTxnIDX
    69  
    70  	TBLInsertIDX
    71  	TBLInsertTxnIDX
    72  	TBLDeleteIDX
    73  	TBLDeleteTxnIDX
    74  	TBLColInsertIDX
    75  	TBLColDeleteIDX
    76  
    77  	SEGInsertIDX
    78  	SEGInsertTxnIDX
    79  	SEGDeleteIDX
    80  	SEGDeleteTxnIDX
    81  
    82  	BLKMetaInsertIDX
    83  	BLKMetaInsertTxnIDX
    84  	BLKMetaDeleteIDX
    85  	BLKMetaDeleteTxnIDX
    86  
    87  	BLKTNMetaInsertIDX
    88  	BLKTNMetaInsertTxnIDX
    89  	BLKTNMetaDeleteIDX
    90  	BLKTNMetaDeleteTxnIDX
    91  
    92  	BLKCNMetaInsertIDX
    93  
    94  	TNMetaIDX
    95  
    96  	// supporting `show accounts` by recording extra
    97  	// account related info in checkpoint
    98  
    99  	StorageUsageInsIDX
   100  
   101  	ObjectInfoIDX
   102  	TNObjectInfoIDX
   103  
   104  	StorageUsageDelIDX
   105  )
   106  
   107  const MaxIDX = StorageUsageDelIDX + 1
   108  
   109  const (
   110  	Checkpoint_Meta_TID_IDX                 = 2
   111  	Checkpoint_Meta_Insert_Block_LOC_IDX    = 3
   112  	Checkpoint_Meta_CN_Delete_Block_LOC_IDX = 4
   113  	Checkpoint_Meta_Delete_Block_LOC_IDX    = 5
   114  	Checkpoint_Meta_Object_LOC_IDX          = 6
   115  	Checkpoint_Meta_Usage_Ins_LOC_IDX       = 7
   116  	Checkpoint_Meta_Usage_Del_LOC_IDX       = 8
   117  )
   118  
   119  // for ver1-3
   120  const (
   121  	Checkpoint_Meta_Insert_Block_Start_IDX = 3
   122  	Checkpoint_Meta_Insert_Block_End_IDX   = 4
   123  	Checkpoint_Meta_Delete_Block_Start_IDX = 5
   124  	Checkpoint_Meta_Delete_Block_End_IDX   = 6
   125  	Checkpoint_Meta_Object_Start_IDX       = 7
   126  	Checkpoint_Meta_Object_End_IDX         = 8
   127  )
   128  
   129  type checkpointDataItem struct {
   130  	schema *catalog.Schema
   131  	types  []types.Type
   132  	attrs  []string
   133  }
   134  
   135  var checkpointDataSchemas_V1 [MaxIDX]*catalog.Schema
   136  var checkpointDataSchemas_V2 [MaxIDX]*catalog.Schema
   137  var checkpointDataSchemas_V3 [MaxIDX]*catalog.Schema
   138  var checkpointDataSchemas_V4 [MaxIDX]*catalog.Schema
   139  var checkpointDataSchemas_V5 [MaxIDX]*catalog.Schema
   140  var checkpointDataSchemas_V6 [MaxIDX]*catalog.Schema
   141  var checkpointDataSchemas_V7 [MaxIDX]*catalog.Schema
   142  var checkpointDataSchemas_V8 [MaxIDX]*catalog.Schema
   143  var checkpointDataSchemas_V9 [MaxIDX]*catalog.Schema
   144  var checkpointDataSchemas_V10 [MaxIDX]*catalog.Schema
   145  var checkpointDataSchemas_V11 [MaxIDX]*catalog.Schema
   146  var checkpointDataSchemas_Curr [MaxIDX]*catalog.Schema
   147  
   148  var checkpointDataReferVersions map[uint32][MaxIDX]*checkpointDataItem
   149  
   150  func init() {
   151  	checkpointDataSchemas_V1 = [MaxIDX]*catalog.Schema{
   152  		MetaSchema_V1,
   153  		catalog.SystemDBSchema,
   154  		TxnNodeSchema,
   155  		DelSchema, // 3
   156  		DBTNSchema,
   157  		catalog.SystemTableSchema_V1,
   158  		TblTNSchema,
   159  		DelSchema, // 7
   160  		TblTNSchema,
   161  		catalog.SystemColumnSchema_V1,
   162  		DelSchema,
   163  		SegSchema, // 11
   164  		SegTNSchema,
   165  		DelSchema,
   166  		SegTNSchema,
   167  		BlkMetaSchema_V1, // 15
   168  		BlkTNSchema,
   169  		DelSchema,
   170  		BlkTNSchema,
   171  		BlkMetaSchema_V1, // 19
   172  		BlkTNSchema,
   173  		DelSchema,
   174  		BlkTNSchema,
   175  		BlkMetaSchema_V1, // 23
   176  		TNMetaSchema,
   177  		StorageUsageSchema, // 25
   178  		ObjectInfoSchema,
   179  		ObjectInfoSchema, // 27
   180  		StorageUsageSchema,
   181  	}
   182  	checkpointDataSchemas_V2 = [MaxIDX]*catalog.Schema{
   183  		MetaSchema_V1,
   184  		catalog.SystemDBSchema,
   185  		TxnNodeSchema,
   186  		DelSchema, // 3
   187  		DBTNSchema,
   188  		catalog.SystemTableSchema,
   189  		TblTNSchema,
   190  		DelSchema, // 7
   191  		TblTNSchema,
   192  		catalog.SystemColumnSchema_V1,
   193  		DelSchema,
   194  		SegSchema, // 11
   195  		SegTNSchema,
   196  		DelSchema,
   197  		SegTNSchema,
   198  		BlkMetaSchema_V1, // 15
   199  		BlkTNSchema,
   200  		DelSchema,
   201  		BlkTNSchema,
   202  		BlkMetaSchema_V1, // 19
   203  		BlkTNSchema,
   204  		DelSchema,
   205  		BlkTNSchema,
   206  		BlkMetaSchema_V1, // 23
   207  		TNMetaSchema,
   208  		StorageUsageSchema, // 25
   209  		ObjectInfoSchema,
   210  		ObjectInfoSchema, // 27
   211  		StorageUsageSchema,
   212  	}
   213  	checkpointDataSchemas_V3 = [MaxIDX]*catalog.Schema{
   214  		MetaSchema_V1,
   215  		catalog.SystemDBSchema,
   216  		TxnNodeSchema,
   217  		DBDelSchema, // 3
   218  		DBTNSchema,
   219  		catalog.SystemTableSchema,
   220  		TblTNSchema,
   221  		TblDelSchema, // 7
   222  		TblTNSchema,
   223  		catalog.SystemColumnSchema_V1,
   224  		ColumnDelSchema,
   225  		SegSchema, // 11
   226  		SegTNSchema,
   227  		DelSchema,
   228  		SegTNSchema,
   229  		BlkMetaSchema_V1, // 15
   230  		BlkTNSchema,
   231  		DelSchema,
   232  		BlkTNSchema,
   233  		BlkMetaSchema_V1, // 19
   234  		BlkTNSchema,
   235  		DelSchema,
   236  		BlkTNSchema,
   237  		BlkMetaSchema_V1, // 23
   238  		TNMetaSchema,
   239  		StorageUsageSchema, // 25
   240  		ObjectInfoSchema,
   241  		ObjectInfoSchema, // 27
   242  		StorageUsageSchema,
   243  	}
   244  	checkpointDataSchemas_V4 = [MaxIDX]*catalog.Schema{
   245  		MetaSchema_V1,
   246  		catalog.SystemDBSchema,
   247  		TxnNodeSchema,
   248  		DBDelSchema, // 3
   249  		DBTNSchema,
   250  		catalog.SystemTableSchema,
   251  		TblTNSchema,
   252  		TblDelSchema, // 7
   253  		TblTNSchema,
   254  		catalog.SystemColumnSchema,
   255  		ColumnDelSchema,
   256  		SegSchema, // 11
   257  		SegTNSchema,
   258  		DelSchema,
   259  		SegTNSchema,
   260  		BlkMetaSchema_V1, // 15
   261  		BlkTNSchema,
   262  		DelSchema,
   263  		BlkTNSchema,
   264  		BlkMetaSchema_V1, // 19
   265  		BlkTNSchema,
   266  		DelSchema,
   267  		BlkTNSchema,
   268  		BlkMetaSchema_V1, // 23
   269  		TNMetaSchema,
   270  		StorageUsageSchema, // 25
   271  		ObjectInfoSchema,
   272  		ObjectInfoSchema, // 27
   273  		StorageUsageSchema,
   274  	}
   275  	checkpointDataSchemas_V5 = [MaxIDX]*catalog.Schema{
   276  		MetaSchema,
   277  		catalog.SystemDBSchema,
   278  		TxnNodeSchema,
   279  		DBDelSchema, // 3
   280  		DBTNSchema,
   281  		catalog.SystemTableSchema,
   282  		TblTNSchema,
   283  		TblDelSchema, // 7
   284  		TblTNSchema,
   285  		catalog.SystemColumnSchema,
   286  		ColumnDelSchema,
   287  		SegSchema, // 11
   288  		SegTNSchema,
   289  		DelSchema,
   290  		SegTNSchema,
   291  		BlkMetaSchema_V1, // 15
   292  		BlkTNSchema,
   293  		DelSchema,
   294  		BlkTNSchema,
   295  		BlkMetaSchema_V1, // 19
   296  		BlkTNSchema,
   297  		DelSchema,
   298  		BlkTNSchema,
   299  		BlkMetaSchema_V1, // 23
   300  		TNMetaSchema,
   301  		StorageUsageSchema, // 25
   302  		ObjectInfoSchema,
   303  		ObjectInfoSchema, // 27
   304  		StorageUsageSchema,
   305  	}
   306  
   307  	checkpointDataSchemas_V6 = [MaxIDX]*catalog.Schema{
   308  		MetaSchema,
   309  		catalog.SystemDBSchema,
   310  		TxnNodeSchema,
   311  		DBDelSchema, // 3
   312  		DBTNSchema,
   313  		catalog.SystemTableSchema,
   314  		TblTNSchema,
   315  		TblDelSchema, // 7
   316  		TblTNSchema,
   317  		catalog.SystemColumnSchema,
   318  		ColumnDelSchema,
   319  		SegSchema, // 11
   320  		SegTNSchema,
   321  		DelSchema,
   322  		SegTNSchema,
   323  		BlkMetaSchema, // 15
   324  		BlkTNSchema,
   325  		DelSchema,
   326  		BlkTNSchema,
   327  		BlkMetaSchema, // 19
   328  		BlkTNSchema,
   329  		DelSchema,
   330  		BlkTNSchema,
   331  		BlkMetaSchema, // 23
   332  		TNMetaSchema,
   333  		StorageUsageSchema, // 25
   334  		ObjectInfoSchema,
   335  		ObjectInfoSchema,
   336  		StorageUsageSchema,
   337  	}
   338  	// Checkpoint V7, V8 update checkpoint metadata
   339  	checkpointDataSchemas_V7 = checkpointDataSchemas_V6
   340  	checkpointDataSchemas_V8 = checkpointDataSchemas_V6
   341  
   342  	// adding extra batches in V9, recording the blk rows and size
   343  	// changing and the account, db, table info the blk belongs to.
   344  	// this enabled the optimization of `show accounts` in CN side.
   345  	checkpointDataSchemas_V9 = [MaxIDX]*catalog.Schema{
   346  		MetaSchema,
   347  		catalog.SystemDBSchema,
   348  		TxnNodeSchema,
   349  		DBDelSchema, // 3
   350  		DBTNSchema,
   351  		catalog.SystemTableSchema,
   352  		TblTNSchema,
   353  		TblDelSchema, // 7
   354  		TblTNSchema,
   355  		catalog.SystemColumnSchema,
   356  		ColumnDelSchema,
   357  		SegSchema, // 11
   358  		SegTNSchema,
   359  		DelSchema,
   360  		SegTNSchema,
   361  		BlkMetaSchema, // 15
   362  		BlkTNSchema,
   363  		DelSchema,
   364  		BlkTNSchema,
   365  		BlkMetaSchema, // 19
   366  		BlkTNSchema,
   367  		DelSchema,
   368  		BlkTNSchema,
   369  		BlkMetaSchema, // 23
   370  		TNMetaSchema,
   371  		StorageUsageSchema, // 25
   372  		ObjectInfoSchema,
   373  		ObjectInfoSchema,
   374  		StorageUsageSchema,
   375  	}
   376  	// version 10 add objectinfo
   377  	checkpointDataSchemas_V10 = checkpointDataSchemas_V9
   378  
   379  	// v11 add storage usage del bat
   380  	checkpointDataSchemas_V11 = [MaxIDX]*catalog.Schema{
   381  		MetaSchema,
   382  		catalog.SystemDBSchema,
   383  		TxnNodeSchema,
   384  		DBDelSchema, // 3
   385  		DBTNSchema,
   386  		catalog.SystemTableSchema,
   387  		TblTNSchema,
   388  		TblDelSchema, // 7
   389  		TblTNSchema,
   390  		catalog.SystemColumnSchema,
   391  		ColumnDelSchema,
   392  		SegSchema, // 11
   393  		SegTNSchema,
   394  		DelSchema,
   395  		SegTNSchema,
   396  		BlkMetaSchema, // 15
   397  		BlkTNSchema,
   398  		DelSchema,
   399  		BlkTNSchema,
   400  		BlkMetaSchema, // 19
   401  		BlkTNSchema,
   402  		DelSchema,
   403  		BlkTNSchema,
   404  		BlkMetaSchema, // 23
   405  		TNMetaSchema,
   406  		StorageUsageSchema, // 25
   407  		ObjectInfoSchema,
   408  		ObjectInfoSchema,
   409  		StorageUsageSchema,
   410  	}
   411  
   412  	checkpointDataReferVersions = make(map[uint32][MaxIDX]*checkpointDataItem)
   413  
   414  	registerCheckpointDataReferVersion(CheckpointVersion1, checkpointDataSchemas_V1[:])
   415  	registerCheckpointDataReferVersion(CheckpointVersion2, checkpointDataSchemas_V2[:])
   416  	registerCheckpointDataReferVersion(CheckpointVersion3, checkpointDataSchemas_V3[:])
   417  	registerCheckpointDataReferVersion(CheckpointVersion4, checkpointDataSchemas_V4[:])
   418  	registerCheckpointDataReferVersion(CheckpointVersion5, checkpointDataSchemas_V5[:])
   419  	registerCheckpointDataReferVersion(CheckpointVersion6, checkpointDataSchemas_V6[:])
   420  	registerCheckpointDataReferVersion(CheckpointVersion7, checkpointDataSchemas_V7[:])
   421  	registerCheckpointDataReferVersion(CheckpointVersion8, checkpointDataSchemas_V8[:])
   422  	registerCheckpointDataReferVersion(CheckpointVersion9, checkpointDataSchemas_V9[:])
   423  	registerCheckpointDataReferVersion(CheckpointVersion10, checkpointDataSchemas_V10[:])
   424  	registerCheckpointDataReferVersion(CheckpointVersion11, checkpointDataSchemas_V11[:])
   425  	checkpointDataSchemas_Curr = checkpointDataSchemas_V11
   426  }
   427  
   428  func IDXString(idx uint16) string {
   429  	switch idx {
   430  	case MetaIDX:
   431  		return "MetaIDX"
   432  	case DBInsertIDX:
   433  		return "DBInsertIDX"
   434  	case DBInsertTxnIDX:
   435  		return "DBInsertTxnIDX"
   436  	case DBDeleteIDX:
   437  		return "DBDeleteIDX"
   438  	case DBDeleteTxnIDX:
   439  		return "DBDeleteTxnIDX"
   440  	case TBLInsertIDX:
   441  		return "TBLInsertIDX"
   442  	case TBLInsertTxnIDX:
   443  		return "TBLInsertTxnIDX"
   444  	case TBLDeleteIDX:
   445  		return "TBLDeleteIDX"
   446  	case TBLDeleteTxnIDX:
   447  		return "TBLDeleteTxnIDX"
   448  	case TBLColInsertIDX:
   449  		return "TBLColInsertIDX"
   450  	case TBLColDeleteIDX:
   451  		return "TBLColDeleteIDX"
   452  	case SEGInsertIDX:
   453  		return "SEGInsertIDX"
   454  	case SEGInsertTxnIDX:
   455  		return "SEGInsertTxnIDX"
   456  	case SEGDeleteIDX:
   457  		return "SEGDeleteIDX"
   458  	case SEGDeleteTxnIDX:
   459  		return "SEGDeleteTxnIDX"
   460  	case BLKMetaInsertIDX:
   461  		return "BLKMetaInsertIDX"
   462  	case BLKMetaInsertTxnIDX:
   463  		return "BLKMetaInsertTxnIDX"
   464  	case BLKMetaDeleteIDX:
   465  		return "BLKMetaDeleteIDX"
   466  	case BLKMetaDeleteTxnIDX:
   467  		return "BLKMetaDeleteTxnIDX"
   468  
   469  	case BLKTNMetaInsertIDX:
   470  		return "BLKTNMetaInsertIDX"
   471  	case BLKTNMetaInsertTxnIDX:
   472  		return "BLKTNMetaInsertTxnIDX"
   473  	case BLKTNMetaDeleteIDX:
   474  		return "BLKTNMetaDeleteIDX"
   475  	case BLKTNMetaDeleteTxnIDX:
   476  		return "BLKTNMetaDeleteTxnIDX"
   477  
   478  	case BLKCNMetaInsertIDX:
   479  		return "BLKCNMetaInsertIDX"
   480  
   481  	case TNMetaIDX:
   482  		return "TNMetaIDX"
   483  
   484  	case StorageUsageInsIDX:
   485  		return "StorageUsageInsIDX"
   486  
   487  	case ObjectInfoIDX:
   488  		return "ObjectInfoIDX"
   489  	case TNObjectInfoIDX:
   490  		return "TNObjectInfoIDX"
   491  	case StorageUsageDelIDX:
   492  		return "StorageUsageDelIDX"
   493  	default:
   494  		return fmt.Sprintf("UnknownIDX(%d)", idx)
   495  	}
   496  }
   497  
   498  func registerCheckpointDataReferVersion(version uint32, schemas []*catalog.Schema) {
   499  	var checkpointDataRefer [MaxIDX]*checkpointDataItem
   500  	for idx, schema := range schemas {
   501  		checkpointDataRefer[idx] = &checkpointDataItem{
   502  			schema,
   503  			append(BaseTypes, schema.Types()...),
   504  			append(BaseAttr, schema.AllNames()...),
   505  		}
   506  	}
   507  	checkpointDataReferVersions[version] = checkpointDataRefer
   508  }
   509  
   510  func IncrementalCheckpointDataFactory(start, end types.TS, collectUsage bool, skipLoadObjectStats bool) func(c *catalog.Catalog) (*CheckpointData, error) {
   511  	return func(c *catalog.Catalog) (data *CheckpointData, err error) {
   512  		collector := NewIncrementalCollector(start, end, skipLoadObjectStats)
   513  		defer collector.Close()
   514  		err = c.RecurLoop(collector)
   515  		if moerr.IsMoErrCode(err, moerr.OkStopCurrRecur) {
   516  			err = nil
   517  		}
   518  		if err != nil {
   519  			return
   520  		}
   521  		if !skipLoadObjectStats {
   522  			err = collector.LoadAndCollectObject(c, collector.VisitObj)
   523  		}
   524  
   525  		if collectUsage {
   526  			collector.UsageMemo = c.GetUsageMemo().(*TNUsageMemo)
   527  			// collecting usage happens only when do ckp
   528  			FillUsageBatOfIncremental(collector)
   529  		}
   530  
   531  		data = collector.OrphanData()
   532  		return
   533  	}
   534  }
   535  
   536  func BackupCheckpointDataFactory(start, end types.TS) func(c *catalog.Catalog) (*CheckpointData, error) {
   537  	return func(c *catalog.Catalog) (data *CheckpointData, err error) {
   538  		collector := NewBackupCollector(start, end)
   539  		defer collector.Close()
   540  		err = c.RecurLoop(collector)
   541  		if moerr.IsMoErrCode(err, moerr.OkStopCurrRecur) {
   542  			err = nil
   543  		}
   544  		data = collector.OrphanData()
   545  		return
   546  	}
   547  }
   548  
   549  func GlobalCheckpointDataFactory(
   550  	end types.TS,
   551  	versionInterval time.Duration,
   552  ) func(c *catalog.Catalog) (*CheckpointData, error) {
   553  	return func(c *catalog.Catalog) (data *CheckpointData, err error) {
   554  		collector := NewGlobalCollector(end, versionInterval)
   555  		defer collector.Close()
   556  		err = c.RecurLoop(collector)
   557  		if moerr.IsMoErrCode(err, moerr.OkStopCurrRecur) {
   558  			err = nil
   559  		}
   560  
   561  		if err != nil {
   562  			return
   563  		}
   564  		err = collector.LoadAndCollectObject(c, collector.VisitObj)
   565  
   566  		collector.UsageMemo = c.GetUsageMemo().(*TNUsageMemo)
   567  		FillUsageBatOfGlobal(collector)
   568  
   569  		data = collector.OrphanData()
   570  
   571  		return
   572  	}
   573  }
   574  
   575  type BlockLocationsIterator struct {
   576  	offset int
   577  	*BlockLocations
   578  }
   579  
   580  func (i *BlockLocationsIterator) HasNext() bool {
   581  	return i.offset < len(*i.BlockLocations)
   582  }
   583  
   584  func (i *BlockLocationsIterator) Next() BlockLocation {
   585  	loc := (BlockLocation)(*i.BlockLocations)[i.offset : i.offset+BlockLocationLength]
   586  	i.offset += BlockLocationLength
   587  	return loc
   588  }
   589  
   590  type BlockLocations []byte
   591  
   592  func NewEmptyBlockLocations() BlockLocations {
   593  	return make([]byte, 0)
   594  }
   595  
   596  func (l *BlockLocations) Append(loc BlockLocation) {
   597  	*l = append(*l, loc...)
   598  }
   599  
   600  func (l BlockLocations) MakeIterator() *BlockLocationsIterator {
   601  	return &BlockLocationsIterator{
   602  		offset:         0,
   603  		BlockLocations: &l,
   604  	}
   605  }
   606  func (l BlockLocations) String() string {
   607  	s := ""
   608  	it := l.MakeIterator()
   609  	for it.HasNext() {
   610  		loc := it.Next()
   611  		s += loc.String()
   612  	}
   613  	return s
   614  }
   615  
   616  // func (l BlockLocations) append iterator
   617  /*
   618  Location is a fixed-length unmodifiable byte array.
   619  Layout:  Location(objectio.Location) | StartOffset(uint64) | EndOffset(uint64)
   620  */
   621  const (
   622  	LocationOffset      = 0
   623  	LocationLength      = objectio.LocationLen
   624  	StartOffsetOffset   = LocationOffset + LocationLength
   625  	StartOffsetLength   = 8
   626  	EndOffsetOffset     = StartOffsetOffset + StartOffsetLength
   627  	EndOffsetLength     = 8
   628  	BlockLocationLength = EndOffsetOffset + EndOffsetLength
   629  )
   630  
   631  type BlockLocation []byte
   632  
   633  func BuildBlockLoaction(id uint16, start, end uint64) BlockLocation {
   634  	buf := make([]byte, BlockLocationLength)
   635  	copy(buf[StartOffsetOffset:StartOffsetOffset+StartOffsetLength], types.EncodeUint64(&start))
   636  	copy(buf[EndOffsetOffset:EndOffsetOffset+EndOffsetLength], types.EncodeUint64(&end))
   637  	blkLoc := BlockLocation(buf)
   638  	blkLoc.SetID(id)
   639  	return buf
   640  }
   641  func BuildBlockLoactionWithLocation(name objectio.ObjectName, extent objectio.Extent, rows uint32, id uint16, start, end uint64) BlockLocation {
   642  	buf := make([]byte, BlockLocationLength)
   643  	location := objectio.BuildLocation(name, extent, rows, id)
   644  	copy(buf[LocationOffset:LocationOffset+LocationLength], location)
   645  	copy(buf[StartOffsetOffset:StartOffsetOffset+StartOffsetLength], types.EncodeUint64(&start))
   646  	copy(buf[EndOffsetOffset:EndOffsetOffset+EndOffsetLength], types.EncodeUint64(&end))
   647  	return buf
   648  }
   649  func (l BlockLocation) GetID() uint16 {
   650  	return l.GetLocation().ID()
   651  }
   652  func (l BlockLocation) GetLocation() objectio.Location {
   653  	return (objectio.Location)(l[LocationOffset : LocationOffset+LocationLength])
   654  }
   655  func (l BlockLocation) GetStartOffset() uint64 {
   656  	return types.DecodeUint64(l[StartOffsetOffset : StartOffsetOffset+StartOffsetLength])
   657  }
   658  func (l BlockLocation) GetEndOffset() uint64 {
   659  	return types.DecodeUint64(l[EndOffsetOffset : EndOffsetOffset+EndOffsetLength])
   660  }
   661  func (l BlockLocation) SetID(id uint16) {
   662  	l.GetLocation().SetID(id)
   663  }
   664  func (l BlockLocation) SetLocation(location objectio.Location) {
   665  	copy(l[LocationOffset:LocationOffset+LocationLength], location)
   666  }
   667  func (l BlockLocation) SetStartOffset(start uint64) {
   668  	copy(l[StartOffsetOffset:StartOffsetOffset+StartOffsetLength], types.EncodeUint64(&start))
   669  }
   670  func (l BlockLocation) SetEndOffset(end uint64) {
   671  	copy(l[EndOffsetOffset:EndOffsetOffset+EndOffsetLength], types.EncodeUint64(&end))
   672  }
   673  func (l BlockLocation) Contains(i common.ClosedInterval) bool {
   674  	return l.GetStartOffset() <= i.Start && l.GetEndOffset() >= i.End
   675  }
   676  
   677  func (l BlockLocation) String() string {
   678  	if len(l) == 0 {
   679  		return ""
   680  	}
   681  	if len(l) != BlockLocationLength {
   682  		return string(l)
   683  	}
   684  	return fmt.Sprintf("%v_start:%d_end:%d", l.GetLocation().String(), l.GetStartOffset(), l.GetEndOffset())
   685  }
   686  
   687  type TableMeta struct {
   688  	common.ClosedInterval
   689  	locations BlockLocations
   690  }
   691  
   692  const (
   693  	BlockInsert = iota
   694  	BlockDelete
   695  	CNBlockInsert
   696  	ObjectInfo
   697  	StorageUsageIns
   698  	StorageUsageDel
   699  )
   700  
   701  func (m *TableMeta) String() string {
   702  	if m == nil {
   703  		return ""
   704  	}
   705  	return fmt.Sprintf("interval:%v, locations:%v", m.ClosedInterval, m.locations)
   706  }
   707  
   708  const MetaMaxIdx = StorageUsageDel + 1
   709  
   710  type CheckpointMeta struct {
   711  	tables [MetaMaxIdx]*TableMeta
   712  }
   713  
   714  func NewCheckpointMeta() *CheckpointMeta {
   715  	return &CheckpointMeta{}
   716  }
   717  
   718  func NewTableMeta() *TableMeta {
   719  	return &TableMeta{}
   720  }
   721  
   722  func (m *CheckpointMeta) DecodeFromString(keys [][]byte) (err error) {
   723  	for i, key := range keys {
   724  		tableMate := NewTableMeta()
   725  		tableMate.locations = key
   726  		m.tables[i] = tableMate
   727  	}
   728  	return
   729  }
   730  func (m *CheckpointMeta) String() string {
   731  	s := ""
   732  	if m == nil {
   733  		return "nil"
   734  	}
   735  	for idx, table := range m.tables {
   736  		s += fmt.Sprintf("idx:%d, table:%s\n", idx, table)
   737  	}
   738  	return s
   739  }
   740  
   741  type CheckpointData struct {
   742  	meta      map[uint64]*CheckpointMeta
   743  	locations map[string]objectio.Location
   744  	bats      [MaxIDX]*containers.Batch
   745  	allocator *mpool.MPool
   746  }
   747  
   748  func NewCheckpointData(mp *mpool.MPool) *CheckpointData {
   749  	data := &CheckpointData{
   750  		meta:      make(map[uint64]*CheckpointMeta),
   751  		allocator: mp,
   752  	}
   753  	for idx, schema := range checkpointDataSchemas_Curr {
   754  		data.bats[idx] = makeRespBatchFromSchema(schema, mp)
   755  	}
   756  	return data
   757  }
   758  
   759  // for test
   760  func NewCheckpointDataWithVersion(ver uint32, mp *mpool.MPool) *CheckpointData {
   761  	data := &CheckpointData{
   762  		meta:      make(map[uint64]*CheckpointMeta),
   763  		allocator: mp,
   764  	}
   765  
   766  	for idx, item := range checkpointDataReferVersions[ver] {
   767  		data.bats[idx] = makeRespBatchFromSchema(item.schema, mp)
   768  	}
   769  	return data
   770  }
   771  
   772  type BaseCollector struct {
   773  	*catalog.LoopProcessor
   774  	start, end types.TS
   775  
   776  	data                *CheckpointData
   777  	skipLoadObjectStats bool
   778  
   779  	// to prefetch object meta when fill in object info batch
   780  
   781  	// true for prefech object meta
   782  	isPrefetch bool
   783  
   784  	Objects []*catalog.ObjectEntry
   785  	// for storage usage
   786  	Usage struct {
   787  		// db, tbl deletes
   788  		Deletes        []interface{}
   789  		ObjInserts     []*catalog.ObjectEntry
   790  		ObjDeletes     []*catalog.ObjectEntry
   791  		ReservedAccIds map[uint64]struct{}
   792  	}
   793  
   794  	UsageMemo *TNUsageMemo
   795  }
   796  
   797  type IncrementalCollector struct {
   798  	*BaseCollector
   799  }
   800  
   801  func NewIncrementalCollector(start, end types.TS, skipLoadObjectStats bool) *IncrementalCollector {
   802  	collector := &IncrementalCollector{
   803  		BaseCollector: &BaseCollector{
   804  			LoopProcessor:       new(catalog.LoopProcessor),
   805  			data:                NewCheckpointData(common.CheckpointAllocator),
   806  			start:               start,
   807  			end:                 end,
   808  			skipLoadObjectStats: skipLoadObjectStats,
   809  		},
   810  	}
   811  	collector.DatabaseFn = collector.VisitDB
   812  	collector.TableFn = collector.VisitTable
   813  	collector.ObjectFn = collector.VisitObj
   814  	collector.TombstoneFn = collector.VisitTombstone
   815  	return collector
   816  }
   817  
   818  func NewBackupCollector(start, end types.TS) *IncrementalCollector {
   819  	collector := &IncrementalCollector{
   820  		BaseCollector: &BaseCollector{
   821  			LoopProcessor: new(catalog.LoopProcessor),
   822  			data:          NewCheckpointData(common.CheckpointAllocator),
   823  			start:         start,
   824  			end:           end,
   825  		},
   826  	}
   827  	// TODO
   828  	collector.TombstoneFn = collector.VisitTombstone
   829  	collector.ObjectFn = collector.VisitObjForBackup
   830  	return collector
   831  }
   832  
   833  type GlobalCollector struct {
   834  	*BaseCollector
   835  	versionThershold types.TS
   836  }
   837  
   838  func NewGlobalCollector(end types.TS, versionInterval time.Duration) *GlobalCollector {
   839  	versionThresholdTS := types.BuildTS(end.Physical()-versionInterval.Nanoseconds(), end.Logical())
   840  	collector := &GlobalCollector{
   841  		BaseCollector: &BaseCollector{
   842  			LoopProcessor: new(catalog.LoopProcessor),
   843  			data:          NewCheckpointData(common.CheckpointAllocator),
   844  			end:           end,
   845  		},
   846  		versionThershold: versionThresholdTS,
   847  	}
   848  
   849  	collector.DatabaseFn = collector.VisitDB
   850  	collector.TableFn = collector.VisitTable
   851  	collector.ObjectFn = collector.VisitObj
   852  	collector.TombstoneFn = collector.VisitTombstone
   853  
   854  	collector.Usage.ReservedAccIds = make(map[uint64]struct{})
   855  
   856  	return collector
   857  }
   858  
   859  func (data *CheckpointData) ApplyReplayTo(
   860  	c *catalog.Catalog,
   861  	dataFactory catalog.DataFactory,
   862  ) (err error) {
   863  	c.OnReplayDatabaseBatch(data.GetDBBatchs())
   864  	ins, colins, tnins, del, tndel := data.GetTblBatchs()
   865  	c.OnReplayTableBatch(ins, colins, tnins, del, tndel, dataFactory)
   866  	objectInfo := data.GetTNObjectBatchs()
   867  	c.OnReplayObjectBatch(objectInfo, dataFactory)
   868  	objectInfo = data.GetObjectBatchs()
   869  	c.OnReplayObjectBatch(objectInfo, dataFactory)
   870  	ins, tnins, del, tndel = data.GetTNBlkBatchs()
   871  	c.OnReplayBlockBatch(ins, tnins, del, tndel, dataFactory)
   872  	ins, tnins, del, tndel = data.GetBlkBatchs()
   873  	c.OnReplayBlockBatch(ins, tnins, del, tndel, dataFactory)
   874  	return
   875  }
   876  
   877  type CNCheckpointData struct {
   878  	meta map[uint64]*CheckpointMeta
   879  	bats [MaxIDX]*batch.Batch
   880  }
   881  
   882  func NewCNCheckpointData() *CNCheckpointData {
   883  	return &CNCheckpointData{
   884  		meta: make(map[uint64]*CheckpointMeta),
   885  	}
   886  }
   887  
   888  // checkpoint table meta idx to ckp batch idx
   889  func switchCheckpointIdx(i uint16, tableID uint64) uint16 {
   890  	idx := uint16(i)
   891  
   892  	if i == BlockInsert {
   893  		idx = BLKMetaInsertIDX
   894  	} else if i == BlockDelete {
   895  		idx = BLKMetaDeleteIDX
   896  	} else if i == CNBlockInsert {
   897  		idx = BLKCNMetaInsertIDX
   898  	} else if i == ObjectInfo {
   899  		idx = ObjectInfoIDX
   900  	} else if i == StorageUsageIns {
   901  		idx = StorageUsageInsIDX
   902  	} else if i == StorageUsageDel {
   903  		idx = StorageUsageDelIDX
   904  	}
   905  	switch tableID {
   906  	case pkgcatalog.MO_DATABASE_ID:
   907  		if i == BlockInsert {
   908  			idx = DBInsertIDX
   909  		} else if i == BlockDelete {
   910  			idx = DBDeleteIDX
   911  		}
   912  	case pkgcatalog.MO_TABLES_ID:
   913  		if i == BlockInsert {
   914  			idx = TBLInsertIDX
   915  		} else if i == BlockDelete {
   916  			idx = TBLDeleteIDX
   917  		}
   918  	case pkgcatalog.MO_COLUMNS_ID:
   919  		if i == BlockInsert {
   920  			idx = TBLColInsertIDX
   921  		} else if i == BlockDelete {
   922  			idx = TBLColDeleteIDX
   923  		}
   924  	}
   925  	return idx
   926  }
   927  
   928  func (data *CNCheckpointData) InitMetaIdx(
   929  	ctx context.Context, version uint32, reader *blockio.BlockReader,
   930  	location objectio.Location, m *mpool.MPool,
   931  ) error {
   932  	if data.bats[MetaIDX] == nil {
   933  		metaIdx := checkpointDataReferVersions[version][MetaIDX]
   934  		metaBats, err := LoadCNSubBlkColumnsByMeta(version, ctx, metaIdx.types, metaIdx.attrs, MetaIDX, reader, m)
   935  		if err != nil {
   936  			return err
   937  		}
   938  		data.bats[MetaIDX] = metaBats[0]
   939  		if version < CheckpointVersion5 {
   940  			err = data.fillInMetaBatchWithLocation(location, m)
   941  			if err != nil {
   942  				return err
   943  			}
   944  		}
   945  	}
   946  	return nil
   947  }
   948  
   949  func (data *CNCheckpointData) PrefetchMetaIdx(
   950  	ctx context.Context,
   951  	version uint32,
   952  	idxes []uint16,
   953  	key objectio.Location,
   954  	service fileservice.FileService,
   955  ) (err error) {
   956  	var pref blockio.PrefetchParams
   957  	pref, err = blockio.BuildPrefetchParams(service, key)
   958  	if err != nil {
   959  		return
   960  	}
   961  	pref.AddBlockWithType(idxes, []uint16{0}, uint16(objectio.ConvertToSchemaType(MetaIDX)))
   962  
   963  	return blockio.PrefetchWithMerged(pref)
   964  }
   965  
   966  func (data *CNCheckpointData) PrefetchMetaFrom(
   967  	ctx context.Context,
   968  	version uint32,
   969  	location objectio.Location,
   970  	service fileservice.FileService,
   971  	tableID uint64) (err error) {
   972  	meta := data.GetTableMeta(tableID, version, location)
   973  	if meta == nil {
   974  		return
   975  	}
   976  	locations := make(map[string]objectio.Location)
   977  	for _, table := range meta.tables {
   978  		if table == nil {
   979  			continue
   980  		}
   981  		it := table.locations.MakeIterator()
   982  		for it.HasNext() {
   983  			block := it.Next()
   984  			name := block.GetLocation().Name().String()
   985  			if locations[name] != nil {
   986  				locations[name] = block.GetLocation()
   987  			}
   988  		}
   989  	}
   990  	for _, location := range locations {
   991  		err = blockio.PrefetchMeta(service, location)
   992  	}
   993  	return err
   994  }
   995  
   996  func (data *CNCheckpointData) PrefetchFrom(
   997  	ctx context.Context,
   998  	version uint32,
   999  	service fileservice.FileService,
  1000  	key objectio.Location,
  1001  	tableID uint64) (err error) {
  1002  	// if version < CheckpointVersion4 {
  1003  	// 	return prefetchCheckpointData(ctx, version, service, key)
  1004  	// }
  1005  	meta := data.GetTableMeta(tableID, version, key)
  1006  	if meta == nil {
  1007  		return
  1008  	}
  1009  	// for ver less than 5, some tablemeta is empty
  1010  	empty := true
  1011  	files := make(map[string]*blockio.PrefetchParams)
  1012  	for i, table := range meta.tables {
  1013  		if table == nil {
  1014  			continue
  1015  		}
  1016  		if i > BlockDelete {
  1017  			if tableID == pkgcatalog.MO_DATABASE_ID ||
  1018  				tableID == pkgcatalog.MO_TABLES_ID ||
  1019  				tableID == pkgcatalog.MO_COLUMNS_ID {
  1020  				break
  1021  			}
  1022  		}
  1023  		if i == ObjectInfo && version < CheckpointVersion10 {
  1024  			continue
  1025  		}
  1026  		idx := switchCheckpointIdx(uint16(i), tableID)
  1027  		schema := checkpointDataReferVersions[version][uint32(idx)]
  1028  		idxes := make([]uint16, len(schema.attrs))
  1029  		for attr := range schema.attrs {
  1030  			idxes[attr] = uint16(attr)
  1031  		}
  1032  		it := table.locations.MakeIterator()
  1033  		for it.HasNext() {
  1034  			block := it.Next()
  1035  			location := block.GetLocation()
  1036  			if files[location.Name().String()] == nil {
  1037  				var pref blockio.PrefetchParams
  1038  				pref, err = blockio.BuildPrefetchParams(service, location)
  1039  				if err != nil {
  1040  					return
  1041  				}
  1042  				files[location.Name().String()] = &pref
  1043  			}
  1044  			pref := *files[location.Name().String()]
  1045  			pref.AddBlockWithType(idxes, []uint16{block.GetID()}, uint16(objectio.ConvertToSchemaType(idx)))
  1046  			empty = false
  1047  		}
  1048  	}
  1049  	if empty {
  1050  		return
  1051  	}
  1052  	for _, pref := range files {
  1053  		err = blockio.PrefetchWithMerged(*pref)
  1054  		if err != nil {
  1055  			return
  1056  		}
  1057  	}
  1058  	return nil
  1059  }
  1060  
  1061  func (data *CNCheckpointData) isMOCatalogTables(tid uint64) bool {
  1062  	return tid == pkgcatalog.MO_DATABASE_ID || tid == pkgcatalog.MO_TABLES_ID || tid == pkgcatalog.MO_COLUMNS_ID
  1063  }
  1064  func (data *CNCheckpointData) GetTableMeta(tableID uint64, version uint32, loc objectio.Location) (meta *CheckpointMeta) {
  1065  	if len(data.meta) != 0 {
  1066  		meta = data.meta[tableID]
  1067  		return
  1068  	}
  1069  	if version <= CheckpointVersion4 && data.isMOCatalogTables(tableID) {
  1070  		tableMeta := NewCheckpointMeta()
  1071  		switch tableID {
  1072  		case pkgcatalog.MO_DATABASE_ID:
  1073  			insertTableMeta := NewTableMeta()
  1074  			insertBlockLoc := BuildBlockLoactionWithLocation(loc.Name(), loc.Extent(), 0, DBInsertIDX, 0, 0)
  1075  			insertTableMeta.locations = make([]byte, 0)
  1076  			insertTableMeta.locations.Append(insertBlockLoc)
  1077  			tableMeta.tables[BlockInsert] = insertTableMeta
  1078  
  1079  			deleteTableMeta := NewTableMeta()
  1080  			deleteBlockLoc := BuildBlockLoactionWithLocation(loc.Name(), loc.Extent(), 0, DBDeleteIDX, 0, 0)
  1081  			deleteTableMeta.locations = make([]byte, 0)
  1082  			deleteTableMeta.locations.Append(deleteBlockLoc)
  1083  			tableMeta.tables[BlockDelete] = deleteTableMeta
  1084  		case pkgcatalog.MO_TABLES_ID:
  1085  			insertTableMeta := NewTableMeta()
  1086  			insertBlockLoc := BuildBlockLoactionWithLocation(loc.Name(), loc.Extent(), 0, TBLInsertIDX, 0, 0)
  1087  			insertTableMeta.locations = make([]byte, 0)
  1088  			insertTableMeta.locations.Append(insertBlockLoc)
  1089  			tableMeta.tables[BlockInsert] = insertTableMeta
  1090  
  1091  			deleteTableMeta := NewTableMeta()
  1092  			deleteBlockLoc := BuildBlockLoactionWithLocation(loc.Name(), loc.Extent(), 0, TBLDeleteIDX, 0, 0)
  1093  			deleteTableMeta.locations = make([]byte, 0)
  1094  			deleteTableMeta.locations.Append(deleteBlockLoc)
  1095  			tableMeta.tables[BlockDelete] = deleteTableMeta
  1096  		case pkgcatalog.MO_COLUMNS_ID:
  1097  			insertTableMeta := NewTableMeta()
  1098  			insertBlockLoc := BuildBlockLoactionWithLocation(loc.Name(), loc.Extent(), 0, TBLColInsertIDX, 0, 0)
  1099  			insertTableMeta.locations = make([]byte, 0)
  1100  			insertTableMeta.locations.Append(insertBlockLoc)
  1101  			tableMeta.tables[BlockInsert] = insertTableMeta
  1102  
  1103  			deleteTableMeta := NewTableMeta()
  1104  			deleteBlockLoc := BuildBlockLoactionWithLocation(loc.Name(), loc.Extent(), 0, TBLColDeleteIDX, 0, 0)
  1105  			deleteTableMeta.locations = make([]byte, 0)
  1106  			deleteTableMeta.locations.Append(deleteBlockLoc)
  1107  			tableMeta.tables[BlockDelete] = deleteTableMeta
  1108  		}
  1109  		return tableMeta
  1110  	}
  1111  	tidVec := vector.MustFixedCol[uint64](data.bats[MetaIDX].Vecs[Checkpoint_Meta_TID_IDX])
  1112  	blkIns := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Insert_Block_LOC_IDX]
  1113  	blkCNIns := data.bats[MetaIDX].Vecs[Checkpoint_Meta_CN_Delete_Block_LOC_IDX]
  1114  	blkDel := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Delete_Block_LOC_IDX]
  1115  	segDel := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Object_LOC_IDX]
  1116  
  1117  	var usageInsVec, usageDelVec *vector.Vector
  1118  	if version >= CheckpointVersion11 {
  1119  		usageInsVec = data.bats[MetaIDX].Vecs[Checkpoint_Meta_Usage_Ins_LOC_IDX]
  1120  		usageDelVec = data.bats[MetaIDX].Vecs[Checkpoint_Meta_Usage_Del_LOC_IDX]
  1121  	}
  1122  
  1123  	var i int
  1124  	if version <= CheckpointVersion4 {
  1125  		i = -1
  1126  		for idx, id := range tidVec {
  1127  			if id == tableID {
  1128  				i = idx
  1129  			}
  1130  		}
  1131  		if i < 0 {
  1132  			return
  1133  		}
  1134  	} else {
  1135  		i = vector.OrderedFindFirstIndexInSortedSlice[uint64](tableID, tidVec)
  1136  		if i < 0 {
  1137  			return
  1138  		}
  1139  	}
  1140  	tid := tidVec[i]
  1141  	blkInsStr := blkIns.GetBytesAt(i)
  1142  	blkCNInsStr := blkCNIns.GetBytesAt(i)
  1143  	blkDelStr := blkDel.GetBytesAt(i)
  1144  	segDelStr := segDel.GetBytesAt(i)
  1145  	tableMeta := NewCheckpointMeta()
  1146  	if len(blkInsStr) > 0 {
  1147  		blkInsertTableMeta := NewTableMeta()
  1148  		blkInsertTableMeta.locations = blkInsStr
  1149  		// blkInsertOffset
  1150  		tableMeta.tables[BlockInsert] = blkInsertTableMeta
  1151  	}
  1152  	if len(blkCNInsStr) > 0 {
  1153  		blkDeleteTableMeta := NewTableMeta()
  1154  		blkDeleteTableMeta.locations = blkDelStr
  1155  		tableMeta.tables[BlockDelete] = blkDeleteTableMeta
  1156  		cnBlkInsTableMeta := NewTableMeta()
  1157  		cnBlkInsTableMeta.locations = blkCNInsStr
  1158  		tableMeta.tables[CNBlockInsert] = cnBlkInsTableMeta
  1159  	} else {
  1160  		if tableID == pkgcatalog.MO_DATABASE_ID ||
  1161  			tableID == pkgcatalog.MO_TABLES_ID ||
  1162  			tableID == pkgcatalog.MO_COLUMNS_ID {
  1163  			if len(blkDelStr) > 0 {
  1164  				blkDeleteTableMeta := NewTableMeta()
  1165  				blkDeleteTableMeta.locations = blkDelStr
  1166  				tableMeta.tables[BlockDelete] = blkDeleteTableMeta
  1167  			}
  1168  		}
  1169  	}
  1170  	if len(segDelStr) > 0 {
  1171  		segDeleteTableMeta := NewTableMeta()
  1172  		segDeleteTableMeta.locations = segDelStr
  1173  		tableMeta.tables[ObjectInfo] = segDeleteTableMeta
  1174  	}
  1175  
  1176  	if usageInsVec != nil {
  1177  		usageInsTableMeta := NewTableMeta()
  1178  		usageDelTableMeta := NewTableMeta()
  1179  
  1180  		usageInsTableMeta.locations = usageInsVec.GetBytesAt(i)
  1181  		usageDelTableMeta.locations = usageDelVec.GetBytesAt(i)
  1182  
  1183  		tableMeta.tables[StorageUsageIns] = usageInsTableMeta
  1184  		tableMeta.tables[StorageUsageDel] = usageDelTableMeta
  1185  	}
  1186  
  1187  	data.meta[tid] = tableMeta
  1188  	meta = data.meta[tableID]
  1189  	return
  1190  }
  1191  func (data *CNCheckpointData) fillInMetaBatchWithLocation(location objectio.Location, m *mpool.MPool) (err error) {
  1192  	length := data.bats[MetaIDX].Vecs[2].Length()
  1193  	insVec := vector.NewVec(types.T_varchar.ToType())
  1194  	cnInsVec := vector.NewVec(types.T_varchar.ToType())
  1195  	delVec := vector.NewVec(types.T_varchar.ToType())
  1196  	segVec := vector.NewVec(types.T_varchar.ToType())
  1197  
  1198  	blkInsStart := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Insert_Block_Start_IDX]
  1199  	blkInsEnd := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Insert_Block_End_IDX]
  1200  	blkDelStart := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Delete_Block_Start_IDX]
  1201  	blkDelEnd := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Delete_Block_End_IDX]
  1202  	segDelStart := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Object_Start_IDX]
  1203  	segDelEnd := data.bats[MetaIDX].Vecs[Checkpoint_Meta_Object_End_IDX]
  1204  	for i := 0; i < length; i++ {
  1205  		insStart := vector.GetFixedAt[int32](blkInsStart, i)
  1206  		insEnd := vector.GetFixedAt[int32](blkInsEnd, i)
  1207  		if insStart < insEnd {
  1208  			insLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKMetaInsertIDX, uint64(insStart), uint64(insEnd))
  1209  			err = vector.AppendAny(insVec, []byte(insLoc), false, m)
  1210  			if err != nil {
  1211  				return
  1212  			}
  1213  		} else {
  1214  			err = vector.AppendAny(insVec, nil, true, m)
  1215  			if err != nil {
  1216  				return
  1217  			}
  1218  		}
  1219  
  1220  		delStart := vector.GetFixedAt[int32](blkDelStart, i)
  1221  		delEnd := vector.GetFixedAt[int32](blkDelEnd, i)
  1222  		if delStart < delEnd {
  1223  			delLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKMetaDeleteIDX, uint64(delStart), uint64(delEnd))
  1224  			err = vector.AppendAny(delVec, []byte(delLoc), false, m)
  1225  			if err != nil {
  1226  				return
  1227  			}
  1228  			cnInsLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKCNMetaInsertIDX, uint64(delStart), uint64(delEnd))
  1229  			err = vector.AppendAny(cnInsVec, []byte(cnInsLoc), false, m)
  1230  			if err != nil {
  1231  				return
  1232  			}
  1233  		} else {
  1234  			err = vector.AppendAny(delVec, nil, true, m)
  1235  			if err != nil {
  1236  				return
  1237  			}
  1238  			err = vector.AppendAny(cnInsVec, nil, true, m)
  1239  			if err != nil {
  1240  				return
  1241  			}
  1242  		}
  1243  
  1244  		objStart := vector.GetFixedAt[int32](segDelStart, i)
  1245  		objEnd := vector.GetFixedAt[int32](segDelEnd, i)
  1246  		if objStart < objEnd {
  1247  			segLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), SEGDeleteIDX, uint64(objStart), uint64(objEnd))
  1248  			err = vector.AppendAny(segVec, []byte(segLoc), false, m)
  1249  			if err != nil {
  1250  				return
  1251  			}
  1252  		} else {
  1253  			err = vector.AppendAny(segVec, nil, true, m)
  1254  			if err != nil {
  1255  				return
  1256  			}
  1257  		}
  1258  	}
  1259  
  1260  	data.bats[MetaIDX].Vecs[Checkpoint_Meta_Insert_Block_LOC_IDX] = insVec
  1261  	data.bats[MetaIDX].Vecs[Checkpoint_Meta_CN_Delete_Block_LOC_IDX] = cnInsVec
  1262  	data.bats[MetaIDX].Vecs[Checkpoint_Meta_Delete_Block_LOC_IDX] = delVec
  1263  	data.bats[MetaIDX].Vecs[Checkpoint_Meta_Object_LOC_IDX] = segVec
  1264  	return
  1265  }
  1266  
  1267  func (data *CNCheckpointData) ReadFromData(
  1268  	ctx context.Context,
  1269  	tableID uint64,
  1270  	location objectio.Location,
  1271  	reader *blockio.BlockReader,
  1272  	version uint32,
  1273  	m *mpool.MPool,
  1274  ) (dataBats []*batch.Batch, err error) {
  1275  	// if err = data.InitMetaIdx(ctx, version, reader,location,m); err != nil {
  1276  	// 	return
  1277  	// }
  1278  	if version <= CheckpointVersion4 {
  1279  		if tableID == pkgcatalog.MO_DATABASE_ID || tableID == pkgcatalog.MO_TABLES_ID || tableID == pkgcatalog.MO_COLUMNS_ID {
  1280  			dataBats = make([]*batch.Batch, MetaMaxIdx)
  1281  			switch tableID {
  1282  			case pkgcatalog.MO_DATABASE_ID:
  1283  				item := checkpointDataReferVersions[version][DBInsertIDX]
  1284  				dataBats[BlockInsert], err = LoadCNSubBlkColumnsByMetaWithId(ctx, item.types, item.attrs, 0, uint16(DBInsertIDX), version, reader, m)
  1285  				if err != nil {
  1286  					return
  1287  				}
  1288  				item = checkpointDataReferVersions[version][DBDeleteIDX]
  1289  				dataBats[BlockDelete], err = LoadCNSubBlkColumnsByMetaWithId(ctx, item.types, item.attrs, 0, uint16(DBDeleteIDX), version, reader, m)
  1290  				if err != nil {
  1291  					return
  1292  				}
  1293  			case pkgcatalog.MO_TABLES_ID:
  1294  				item := checkpointDataReferVersions[version][TBLInsertIDX]
  1295  				dataBats[BlockInsert], err = LoadCNSubBlkColumnsByMetaWithId(ctx, item.types, item.attrs, 0, uint16(TBLInsertIDX), version, reader, m)
  1296  				if err != nil {
  1297  					return
  1298  				}
  1299  				item = checkpointDataReferVersions[version][TBLDeleteIDX]
  1300  				dataBats[BlockDelete], err = LoadCNSubBlkColumnsByMetaWithId(ctx, item.types, item.attrs, 0, uint16(TBLDeleteIDX), version, reader, m)
  1301  				if err != nil {
  1302  					return
  1303  				}
  1304  			case pkgcatalog.MO_COLUMNS_ID:
  1305  				item := checkpointDataReferVersions[version][TBLColInsertIDX]
  1306  				dataBats[BlockInsert], err = LoadCNSubBlkColumnsByMetaWithId(ctx, item.types, item.attrs, 0, uint16(TBLColInsertIDX), version, reader, m)
  1307  				if err != nil {
  1308  					return
  1309  				}
  1310  				item = checkpointDataReferVersions[version][TBLColDeleteIDX]
  1311  				dataBats[BlockDelete], err = LoadCNSubBlkColumnsByMetaWithId(ctx, item.types, item.attrs, 0, uint16(TBLColDeleteIDX), version, reader, m)
  1312  				if err != nil {
  1313  					return
  1314  				}
  1315  			}
  1316  			if version == CheckpointVersion1 {
  1317  				if tableID == pkgcatalog.MO_TABLES_ID {
  1318  					bat := dataBats[BlockInsert]
  1319  					if bat != nil {
  1320  						versionVec := vector.MustFixedCol[uint32](bat.Vecs[pkgcatalog.MO_TABLES_VERSION_IDX+2]) // 2 for rowid and committs
  1321  						length := len(versionVec)
  1322  						vec := vector.NewVec(types.T_uint32.ToType())
  1323  						for i := 0; i < length; i++ {
  1324  							err = vector.AppendFixed[uint32](vec, pkgcatalog.CatalogVersion_V1, false, m)
  1325  							if err != nil {
  1326  								return
  1327  							}
  1328  						}
  1329  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemRelAttr_CatalogVersion)
  1330  						bat.Vecs = append(bat.Vecs, vec)
  1331  					}
  1332  				}
  1333  			}
  1334  			if version <= CheckpointVersion2 {
  1335  				if tableID == pkgcatalog.MO_DATABASE_ID {
  1336  					bat := dataBats[BlockDelete]
  1337  					if bat != nil {
  1338  						rowIDVec := vector.MustFixedCol[types.Rowid](bat.Vecs[0])
  1339  						length := len(rowIDVec)
  1340  						pkVec := vector.NewVec(types.T_uint64.ToType())
  1341  						for i := 0; i < length; i++ {
  1342  							err = vector.AppendFixed[uint64](pkVec, objectio.HackRowidToU64(rowIDVec[i]), false, m)
  1343  							if err != nil {
  1344  								return
  1345  							}
  1346  						}
  1347  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemDBAttr_ID)
  1348  						bat.Vecs = append(bat.Vecs, pkVec)
  1349  					}
  1350  				} else if tableID == pkgcatalog.MO_TABLES_ID {
  1351  					bat := dataBats[BlockDelete]
  1352  					if bat != nil {
  1353  						rowIDVec := vector.MustFixedCol[types.Rowid](bat.Vecs[0])
  1354  						length := len(rowIDVec)
  1355  						pkVec2 := vector.NewVec(types.T_uint64.ToType())
  1356  						for i := 0; i < length; i++ {
  1357  							err = vector.AppendFixed[uint64](pkVec2, objectio.HackRowidToU64(rowIDVec[i]), false, m)
  1358  							if err != nil {
  1359  								return
  1360  							}
  1361  						}
  1362  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemRelAttr_ID)
  1363  						bat.Vecs = append(bat.Vecs, pkVec2)
  1364  					}
  1365  				} else if tableID == pkgcatalog.MO_COLUMNS_ID {
  1366  					bat := dataBats[BlockDelete]
  1367  					if bat != nil {
  1368  						rowIDVec := vector.MustFixedCol[types.Rowid](bat.Vecs[0])
  1369  						length := len(rowIDVec)
  1370  						pkVec2 := vector.NewVec(types.T_varchar.ToType())
  1371  						for i := 0; i < length; i++ {
  1372  							err = vector.AppendAny(pkVec2, nil, true, m)
  1373  							if err != nil {
  1374  								return
  1375  							}
  1376  						}
  1377  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemColAttr_UniqName)
  1378  						bat.Vecs = append(bat.Vecs, pkVec2)
  1379  					}
  1380  				}
  1381  			}
  1382  			if version <= CheckpointVersion3 {
  1383  				if tableID == pkgcatalog.MO_COLUMNS_ID {
  1384  					bat := dataBats[BlockInsert]
  1385  					if bat != nil {
  1386  						rowIDVec := vector.MustFixedCol[types.Rowid](bat.Vecs[0])
  1387  						length := len(rowIDVec)
  1388  						enumVec := vector.NewVec(types.New(types.T_varchar, types.MaxVarcharLen, 0))
  1389  						for i := 0; i < length; i++ {
  1390  							err = vector.AppendAny(enumVec, []byte(""), false, m)
  1391  							if err != nil {
  1392  								return
  1393  							}
  1394  						}
  1395  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemColAttr_EnumValues)
  1396  						bat.Vecs = append(bat.Vecs, enumVec)
  1397  					}
  1398  				}
  1399  			}
  1400  			return
  1401  		}
  1402  	}
  1403  	meta := data.GetTableMeta(tableID, version, location)
  1404  	if meta == nil {
  1405  		return
  1406  	}
  1407  	dataBats = make([]*batch.Batch, MetaMaxIdx)
  1408  	for i, table := range meta.tables {
  1409  		if table == nil {
  1410  			continue
  1411  		}
  1412  		if i > BlockDelete {
  1413  			if tableID == pkgcatalog.MO_DATABASE_ID ||
  1414  				tableID == pkgcatalog.MO_TABLES_ID ||
  1415  				tableID == pkgcatalog.MO_COLUMNS_ID {
  1416  				break
  1417  			}
  1418  		}
  1419  		if i == ObjectInfo && version < CheckpointVersion10 {
  1420  			continue
  1421  		}
  1422  		idx := switchCheckpointIdx(uint16(i), tableID)
  1423  		it := table.locations.MakeIterator()
  1424  		for it.HasNext() {
  1425  			block := it.Next()
  1426  			var bat *batch.Batch
  1427  			schema := checkpointDataReferVersions[version][uint32(idx)]
  1428  			reader, err = blockio.NewObjectReader(reader.GetObjectReader().GetObject().GetFs(), block.GetLocation())
  1429  			if err != nil {
  1430  				return
  1431  			}
  1432  			bat, err = LoadCNSubBlkColumnsByMetaWithId(ctx, schema.types, schema.attrs, uint16(idx), block.GetID(), version, reader, m)
  1433  			if err != nil {
  1434  				return
  1435  			}
  1436  			//logutil.Infof("load block %v: %d-%d to %d", block.GetLocation().String(), block.GetStartOffset(), block.GetEndOffset(), bat.Vecs[0].Length())
  1437  			if block.GetEndOffset() == 0 {
  1438  				continue
  1439  			}
  1440  			windowCNBatch(bat, block.GetStartOffset(), block.GetEndOffset())
  1441  			if dataBats[uint32(i)] == nil {
  1442  				cnBatch := batch.NewWithSize(len(bat.Vecs))
  1443  				cnBatch.Attrs = make([]string, len(bat.Attrs))
  1444  				copy(cnBatch.Attrs, bat.Attrs)
  1445  				for n := range cnBatch.Vecs {
  1446  					cnBatch.Vecs[n] = vector.NewVec(*bat.Vecs[n].GetType())
  1447  					if err = cnBatch.Vecs[n].UnionBatch(bat.Vecs[n], 0, bat.Vecs[n].Length(), nil, m); err != nil {
  1448  						return
  1449  					}
  1450  				}
  1451  				dataBats[uint32(i)] = cnBatch
  1452  			} else {
  1453  				dataBats[uint32(i)], err = dataBats[uint32(i)].Append(ctx, m, bat)
  1454  				if err != nil {
  1455  					return
  1456  				}
  1457  			}
  1458  		}
  1459  
  1460  		if version <= CheckpointVersion5 {
  1461  			if dataBats[i] != nil && (idx == BLKCNMetaInsertIDX || idx == BLKMetaInsertIDX) {
  1462  				blkMetaBat := dataBats[i]
  1463  				committs := blkMetaBat.Vecs[2+pkgcatalog.BLOCKMETA_COMMITTS_IDX]
  1464  				blkMetaBat.Attrs = append(blkMetaBat.Attrs, pkgcatalog.BlockMeta_MemTruncPoint)
  1465  				blkMetaBat.Vecs = append(blkMetaBat.Vecs, committs)
  1466  			}
  1467  		}
  1468  	}
  1469  
  1470  	return
  1471  }
  1472  
  1473  func (data *CNCheckpointData) GetTableDataFromBats(tid uint64, bats []*batch.Batch) (ins, del, cnIns, objInfo *api.Batch, err error) {
  1474  	var insTaeBat, delTaeBat, cnInsTaeBat, objInfoTaeBat *batch.Batch
  1475  	if len(bats) == 0 {
  1476  		return
  1477  	}
  1478  	if tid == pkgcatalog.MO_DATABASE_ID || tid == pkgcatalog.MO_TABLES_ID || tid == pkgcatalog.MO_COLUMNS_ID {
  1479  		insTaeBat = bats[BlockInsert]
  1480  		delTaeBat = bats[BlockDelete]
  1481  		if insTaeBat != nil {
  1482  			ins, err = batch.BatchToProtoBatch(insTaeBat)
  1483  			if err != nil {
  1484  				return
  1485  			}
  1486  		}
  1487  		if delTaeBat != nil {
  1488  			del, err = batch.BatchToProtoBatch(delTaeBat)
  1489  			if err != nil {
  1490  				return
  1491  			}
  1492  		}
  1493  		return
  1494  	}
  1495  
  1496  	insTaeBat = bats[BlockInsert]
  1497  	if insTaeBat != nil {
  1498  		ins, err = batch.BatchToProtoBatch(insTaeBat)
  1499  		if err != nil {
  1500  			return
  1501  		}
  1502  	}
  1503  	delTaeBat = bats[BlockDelete]
  1504  	cnInsTaeBat = bats[CNBlockInsert]
  1505  	if delTaeBat != nil {
  1506  		del, err = batch.BatchToProtoBatch(delTaeBat)
  1507  		if err != nil {
  1508  			return
  1509  		}
  1510  	}
  1511  	if cnInsTaeBat != nil {
  1512  		cnIns, err = batch.BatchToProtoBatch(cnInsTaeBat)
  1513  		if err != nil {
  1514  			return
  1515  		}
  1516  	}
  1517  	objInfoTaeBat = bats[ObjectInfo]
  1518  	if objInfoTaeBat != nil {
  1519  		objInfo, err = batch.BatchToProtoBatch(objInfoTaeBat)
  1520  		if err != nil {
  1521  			return
  1522  		}
  1523  	}
  1524  	return
  1525  }
  1526  
  1527  func (data *CNCheckpointData) GetCloseCB(version uint32, m *mpool.MPool) func() {
  1528  	return func() {
  1529  		if version == CheckpointVersion1 {
  1530  			data.closeVector(TBLInsertIDX, pkgcatalog.MO_TABLES_CATALOG_VERSION_IDX+2, m) // 2 for rowid and committs
  1531  		}
  1532  		if version <= CheckpointVersion2 {
  1533  			data.closeVector(DBDeleteIDX, 2, m)
  1534  			data.closeVector(TBLDeleteIDX, 2, m)
  1535  			data.closeVector(TBLColDeleteIDX, 2, m)
  1536  		}
  1537  		if version <= CheckpointVersion3 {
  1538  			data.closeVector(TBLColInsertIDX, 25, m)
  1539  		}
  1540  		if version <= CheckpointVersion4 {
  1541  			data.closeVector(MetaIDX, Checkpoint_Meta_Insert_Block_LOC_IDX, m)
  1542  			data.closeVector(MetaIDX, Checkpoint_Meta_CN_Delete_Block_LOC_IDX, m)
  1543  			data.closeVector(MetaIDX, Checkpoint_Meta_Delete_Block_LOC_IDX, m)
  1544  			data.closeVector(MetaIDX, Checkpoint_Meta_Object_LOC_IDX, m)
  1545  		}
  1546  	}
  1547  }
  1548  
  1549  func (data *CNCheckpointData) closeVector(batIdx uint16, colIdx int, m *mpool.MPool) {
  1550  	bat := data.bats[batIdx]
  1551  	if bat == nil {
  1552  		return
  1553  	}
  1554  	if len(bat.Vecs) <= colIdx {
  1555  		return
  1556  	}
  1557  	vec := data.bats[batIdx].Vecs[colIdx]
  1558  	vec.Free(m)
  1559  
  1560  }
  1561  
  1562  func windowCNBatch(bat *batch.Batch, start, end uint64) {
  1563  	var err error
  1564  	for i, vec := range bat.Vecs {
  1565  		bat.Vecs[i], err = vec.Window(int(start), int(end))
  1566  		if err != nil {
  1567  			panic(err)
  1568  		}
  1569  	}
  1570  }
  1571  
  1572  func (data *CheckpointData) Allocator() *mpool.MPool { return data.allocator }
  1573  
  1574  func (data *CheckpointData) fillInMetaBatchWithLocation(location objectio.Location) {
  1575  	length := data.bats[MetaIDX].Vecs[2].Length()
  1576  	insVec := containers.MakeVector(types.T_varchar.ToType(), data.allocator)
  1577  	cnInsVec := containers.MakeVector(types.T_varchar.ToType(), data.allocator)
  1578  	delVec := containers.MakeVector(types.T_varchar.ToType(), data.allocator)
  1579  	segVec := containers.MakeVector(types.T_varchar.ToType(), data.allocator)
  1580  
  1581  	tidVec := data.bats[MetaIDX].GetVectorByName(SnapshotAttr_TID)
  1582  	blkInsStart := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockInsertBatchStart).GetDownstreamVector()
  1583  	blkInsEnd := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockInsertBatchEnd).GetDownstreamVector()
  1584  	blkDelStart := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockDeleteBatchStart).GetDownstreamVector()
  1585  	blkDelEnd := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockInsertBatchStart).GetDownstreamVector()
  1586  	segDelStart := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockInsertBatchStart).GetDownstreamVector()
  1587  	segDelEnd := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockInsertBatchStart).GetDownstreamVector()
  1588  	for i := 0; i < length; i++ {
  1589  		insStart := vector.GetFixedAt[int32](blkInsStart, i)
  1590  		insEnd := vector.GetFixedAt[int32](blkInsEnd, i)
  1591  		if insStart < insEnd {
  1592  			insLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKMetaInsertIDX, uint64(insStart), uint64(insEnd))
  1593  			insVec.Append([]byte(insLoc), false)
  1594  		} else {
  1595  			insVec.Append(nil, true)
  1596  		}
  1597  
  1598  		delStart := vector.GetFixedAt[int32](blkDelStart, i)
  1599  		delEnd := vector.GetFixedAt[int32](blkDelEnd, i)
  1600  		if delStart < delEnd {
  1601  			delLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKMetaDeleteIDX, uint64(delStart), uint64(delEnd))
  1602  			delVec.Append([]byte(delLoc), false)
  1603  			cnInsLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKCNMetaInsertIDX, uint64(delStart), uint64(delEnd))
  1604  			cnInsVec.Append([]byte(cnInsLoc), false)
  1605  		} else {
  1606  			delVec.Append(nil, true)
  1607  			cnInsVec.Append(nil, true)
  1608  		}
  1609  
  1610  		segStart := vector.GetFixedAt[int32](segDelStart, i)
  1611  		segEnd := vector.GetFixedAt[int32](segDelEnd, i)
  1612  		if segStart < segEnd {
  1613  			segLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), SEGDeleteIDX, uint64(segStart), uint64(segEnd))
  1614  			segVec.Append([]byte(segLoc), false)
  1615  		} else {
  1616  			segVec.Append(nil, true)
  1617  		}
  1618  	}
  1619  
  1620  	tidVec.Append(uint64(0), false)
  1621  	blkLoc := BuildBlockLoactionWithLocation(location.Name(), location.Extent(), location.Rows(), BLKMetaInsertIDX, uint64(0), uint64(0))
  1622  	insVec.Append([]byte(blkLoc), false)
  1623  	cnInsVec.Append(nil, true)
  1624  	delVec.Append(nil, true)
  1625  	segVec.Append(nil, true)
  1626  
  1627  	data.bats[MetaIDX].AddVector(SnapshotMetaAttr_BlockInsertBatchLocation, insVec)
  1628  	data.bats[MetaIDX].AddVector(SnapshotMetaAttr_BlockCNInsertBatchLocation, cnInsVec)
  1629  	data.bats[MetaIDX].AddVector(SnapshotMetaAttr_BlockDeleteBatchLocation, delVec)
  1630  	data.bats[MetaIDX].AddVector(SnapshotMetaAttr_SegDeleteBatchLocation, segVec)
  1631  }
  1632  
  1633  func (data *CheckpointData) prepareMeta() {
  1634  	bat := data.bats[MetaIDX]
  1635  	blkInsLoc := bat.GetVectorByName(SnapshotMetaAttr_BlockInsertBatchLocation).GetDownstreamVector()
  1636  	blkDelLoc := bat.GetVectorByName(SnapshotMetaAttr_BlockDeleteBatchLocation).GetDownstreamVector()
  1637  	blkCNInsLoc := bat.GetVectorByName(SnapshotMetaAttr_BlockCNInsertBatchLocation).GetDownstreamVector()
  1638  	segDelLoc := bat.GetVectorByName(SnapshotMetaAttr_SegDeleteBatchLocation).GetDownstreamVector()
  1639  	tidVec := bat.GetVectorByName(SnapshotAttr_TID).GetDownstreamVector()
  1640  	usageInsLoc := bat.GetVectorByName(CheckpointMetaAttr_StorageUsageInsLocation).GetDownstreamVector()
  1641  	usageDelLoc := bat.GetVectorByName(CheckpointMetaAttr_StorageUsageDelLocation).GetDownstreamVector()
  1642  
  1643  	sortMeta := make([]int, 0)
  1644  	for tid := range data.meta {
  1645  		sortMeta = append(sortMeta, int(tid))
  1646  	}
  1647  	sort.Ints(sortMeta)
  1648  	for _, tid := range sortMeta {
  1649  		vector.AppendFixed[uint64](tidVec, uint64(tid), false, data.allocator)
  1650  		if data.meta[uint64(tid)].tables[BlockInsert] == nil {
  1651  			vector.AppendBytes(blkInsLoc, nil, true, data.allocator)
  1652  		} else {
  1653  			vector.AppendBytes(blkInsLoc, []byte(data.meta[uint64(tid)].tables[BlockInsert].locations), false, data.allocator)
  1654  		}
  1655  		if data.meta[uint64(tid)].tables[BlockDelete] == nil {
  1656  			vector.AppendBytes(blkDelLoc, nil, true, data.allocator)
  1657  		} else {
  1658  			vector.AppendBytes(blkDelLoc, []byte(data.meta[uint64(tid)].tables[BlockDelete].locations), false, data.allocator)
  1659  		}
  1660  		if data.meta[uint64(tid)].tables[CNBlockInsert] == nil {
  1661  			vector.AppendBytes(blkCNInsLoc, nil, true, data.allocator)
  1662  		} else {
  1663  			vector.AppendBytes(blkCNInsLoc, []byte(data.meta[uint64(tid)].tables[CNBlockInsert].locations), false, data.allocator)
  1664  		}
  1665  		if data.meta[uint64(tid)].tables[ObjectInfo] == nil {
  1666  			vector.AppendBytes(segDelLoc, nil, true, data.allocator)
  1667  		} else {
  1668  			vector.AppendBytes(segDelLoc, []byte(data.meta[uint64(tid)].tables[ObjectInfo].locations), false, data.allocator)
  1669  		}
  1670  
  1671  		if data.meta[uint64(tid)].tables[StorageUsageIns] == nil {
  1672  			vector.AppendBytes(usageInsLoc, nil, true, data.allocator)
  1673  		} else {
  1674  			vector.AppendBytes(usageInsLoc, data.meta[uint64(tid)].tables[StorageUsageIns].locations, false, data.allocator)
  1675  		}
  1676  
  1677  		if data.meta[uint64(tid)].tables[StorageUsageDel] == nil {
  1678  			vector.AppendBytes(usageDelLoc, nil, true, data.allocator)
  1679  		} else {
  1680  			vector.AppendBytes(usageDelLoc, data.meta[uint64(tid)].tables[StorageUsageDel].locations, false, data.allocator)
  1681  		}
  1682  	}
  1683  }
  1684  func (data *CheckpointData) resetObjectMeta() {
  1685  	for _, meta := range data.meta {
  1686  		meta.tables[ObjectInfo] = nil
  1687  	}
  1688  }
  1689  func (data *CheckpointData) updateTableMeta(tid uint64, metaIdx int, start, end int32) {
  1690  	meta, ok := data.meta[tid]
  1691  	if !ok {
  1692  		meta = NewCheckpointMeta()
  1693  		data.meta[tid] = meta
  1694  	}
  1695  	if end > start {
  1696  		if meta.tables[metaIdx] == nil {
  1697  			meta.tables[metaIdx] = NewTableMeta()
  1698  			meta.tables[metaIdx].Start = uint64(start)
  1699  			meta.tables[metaIdx].End = uint64(end)
  1700  		} else {
  1701  			if !meta.tables[metaIdx].TryMerge(common.ClosedInterval{Start: uint64(start), End: uint64(end)}) {
  1702  				panic(fmt.Sprintf("logic error interval %v, start %d, end %d", meta.tables[BlockDelete].ClosedInterval, start, end))
  1703  			}
  1704  		}
  1705  	}
  1706  }
  1707  func (data *CheckpointData) updateMOCatalog(tid uint64, insStart, insEnd, delStart, delEnd int32) {
  1708  	if delEnd <= delStart && insEnd <= insStart {
  1709  		return
  1710  	}
  1711  	data.updateTableMeta(tid, BlockInsert, insStart, insEnd)
  1712  	data.updateTableMeta(tid, BlockDelete, delStart, delEnd)
  1713  }
  1714  func (data *CheckpointData) UpdateBlkMeta(tid uint64, insStart, insEnd, delStart, delEnd int32) {
  1715  	if delEnd <= delStart && insEnd <= insStart {
  1716  		return
  1717  	}
  1718  	data.updateTableMeta(tid, BlockInsert, insStart, insEnd)
  1719  	data.updateTableMeta(tid, BlockDelete, delStart, delEnd)
  1720  	data.updateTableMeta(tid, CNBlockInsert, delStart, delEnd)
  1721  }
  1722  
  1723  func (data *CheckpointData) UpdateSegMeta(tid uint64, delStart, delEnd int32) {
  1724  	if delEnd <= delStart {
  1725  		return
  1726  	}
  1727  	data.updateTableMeta(tid, ObjectInfo, delStart, delEnd)
  1728  }
  1729  
  1730  func (data *CheckpointData) UpdateObjectInsertMeta(tid uint64, delStart, delEnd int32) {
  1731  	if delEnd <= delStart {
  1732  		return
  1733  	}
  1734  	data.resetTableMeta(tid, ObjectInfo, delStart, delEnd)
  1735  }
  1736  
  1737  func (data *CheckpointData) resetTableMeta(tid uint64, metaIdx int, start, end int32) {
  1738  	meta, ok := data.meta[tid]
  1739  	if !ok {
  1740  		meta = NewCheckpointMeta()
  1741  		data.meta[tid] = meta
  1742  	}
  1743  	if end > start {
  1744  		if meta.tables[metaIdx] == nil {
  1745  			meta.tables[metaIdx] = NewTableMeta()
  1746  			meta.tables[metaIdx].Start = uint64(start)
  1747  			meta.tables[metaIdx].End = uint64(end)
  1748  		} else {
  1749  			meta.tables[metaIdx].Start = uint64(start)
  1750  			meta.tables[metaIdx].End = uint64(end)
  1751  			return
  1752  		}
  1753  	}
  1754  }
  1755  
  1756  func (data *CheckpointData) UpdateBlockInsertBlkMeta(tid uint64, insStart, insEnd int32) {
  1757  	if insEnd <= insStart {
  1758  		return
  1759  	}
  1760  	data.resetTableMeta(tid, BlockInsert, insStart, insEnd)
  1761  }
  1762  
  1763  func (data *CheckpointData) UpdateBlockDeleteBlkMeta(tid uint64, insStart, insEnd int32) {
  1764  	if insEnd <= insStart {
  1765  		return
  1766  	}
  1767  	data.resetTableMeta(tid, BlockDelete, insStart, insEnd)
  1768  	data.resetTableMeta(tid, CNBlockInsert, insStart, insEnd)
  1769  }
  1770  
  1771  func (data *CheckpointData) PrintData() {
  1772  	logutil.Info(BatchToString("BLK-META-DEL-BAT", data.bats[BLKMetaDeleteIDX], true))
  1773  	logutil.Info(BatchToString("BLK-META-INS-BAT", data.bats[BLKMetaInsertIDX], true))
  1774  }
  1775  
  1776  func formatBatch(bat *containers.Batch) {
  1777  	length := bat.Length()
  1778  	if length == 0 {
  1779  		for _, vec := range bat.Vecs {
  1780  			if vec.Length() > 0 {
  1781  				length = vec.Length()
  1782  				break
  1783  			}
  1784  		}
  1785  		if length == 0 {
  1786  			return
  1787  		}
  1788  	}
  1789  	for i := range bat.Vecs {
  1790  		if bat.Vecs[i].Length() == 0 {
  1791  			bat.Vecs[i] = containers.NewConstNullVector(*bat.Vecs[i].GetType(), length, common.DefaultAllocator)
  1792  		}
  1793  	}
  1794  }
  1795  
  1796  func (data *CheckpointData) prepareTNMetaBatch(
  1797  	checkpointNames []objectio.ObjectName,
  1798  	objectBlocks [][]objectio.BlockObject,
  1799  	schemaTypes [][]uint16,
  1800  ) {
  1801  	for i, blks := range objectBlocks {
  1802  		for y, blk := range blks {
  1803  			location := objectio.BuildLocation(checkpointNames[i], blk.GetExtent(), 0, blk.GetID())
  1804  			data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_BlockLocation).Append([]byte(location), false)
  1805  			data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_SchemaType).Append(schemaTypes[i][y], false)
  1806  		}
  1807  	}
  1808  }
  1809  
  1810  func (data *CheckpointData) FormatData(mp *mpool.MPool) (err error) {
  1811  	for idx := range data.bats {
  1812  		for i, col := range data.bats[idx].Vecs {
  1813  			vec := col.CloneWindow(0, col.Length(), mp)
  1814  			col.Close()
  1815  			data.bats[idx].Vecs[i] = vec
  1816  		}
  1817  	}
  1818  	data.bats[MetaIDX].Close()
  1819  	data.bats[TNMetaIDX].Close()
  1820  	data.bats[MetaIDX] = makeRespBatchFromSchema(checkpointDataSchemas_Curr[MetaIDX], mp)
  1821  	data.bats[TNMetaIDX] = makeRespBatchFromSchema(checkpointDataSchemas_Curr[TNMetaIDX], mp)
  1822  	for tid := range data.meta {
  1823  		for idx := range data.meta[tid].tables {
  1824  			if data.meta[tid].tables[idx] != nil {
  1825  				location := data.meta[tid].tables[idx].locations.MakeIterator()
  1826  				if location.HasNext() {
  1827  					loc := location.Next()
  1828  					if data.meta[tid].tables[idx].Start == 0 && data.meta[tid].tables[idx].End == 0 {
  1829  						data.meta[tid].tables[idx].Start = loc.GetStartOffset()
  1830  						data.meta[tid].tables[idx].End = loc.GetEndOffset()
  1831  					} else {
  1832  						data.meta[tid].tables[idx].TryMerge(common.ClosedInterval{Start: loc.GetStartOffset(), End: loc.GetEndOffset()})
  1833  					}
  1834  				}
  1835  				data.meta[tid].tables[idx].locations = make([]byte, 0)
  1836  			}
  1837  		}
  1838  	}
  1839  	return
  1840  }
  1841  
  1842  type blockIndexes struct {
  1843  	fileNum uint16
  1844  	indexes *BlockLocation
  1845  }
  1846  
  1847  func (data *CheckpointData) WriteTo(
  1848  	fs fileservice.FileService,
  1849  	blockRows int,
  1850  	checkpointSize int,
  1851  ) (CNLocation, TNLocation objectio.Location, checkpointFiles []string, err error) {
  1852  	checkpointNames := make([]objectio.ObjectName, 1)
  1853  	segmentid := objectio.NewSegmentid()
  1854  	fileNum := uint16(0)
  1855  	name := objectio.BuildObjectName(segmentid, fileNum)
  1856  	writer, err := blockio.NewBlockWriterNew(fs, name, 0, nil)
  1857  	if err != nil {
  1858  		return
  1859  	}
  1860  	checkpointNames[0] = name
  1861  	objectBlocks := make([][]objectio.BlockObject, 0)
  1862  	indexes := make([][]blockIndexes, MaxIDX)
  1863  	schemas := make([][]uint16, 0)
  1864  	schemaTypes := make([]uint16, 0)
  1865  	checkpointFiles = make([]string, 0)
  1866  	var objectSize int
  1867  	for i := range checkpointDataSchemas_Curr {
  1868  		if i == int(MetaIDX) || i == int(TNMetaIDX) {
  1869  			continue
  1870  		}
  1871  		offset := 0
  1872  		formatBatch(data.bats[i])
  1873  		var block objectio.BlockObject
  1874  		var bat *containers.Batch
  1875  		var size int
  1876  		var blks []objectio.BlockObject
  1877  		if objectSize > checkpointSize {
  1878  			fileNum++
  1879  			blks, _, err = writer.Sync(context.Background())
  1880  			if err != nil {
  1881  				return
  1882  			}
  1883  			checkpointFiles = append(checkpointFiles, name.String())
  1884  			name = objectio.BuildObjectName(segmentid, fileNum)
  1885  			writer, err = blockio.NewBlockWriterNew(fs, name, 0, nil)
  1886  			if err != nil {
  1887  				return
  1888  			}
  1889  			checkpointNames = append(checkpointNames, name)
  1890  			objectBlocks = append(objectBlocks, blks)
  1891  			schemas = append(schemas, schemaTypes)
  1892  			schemaTypes = make([]uint16, 0)
  1893  			objectSize = 0
  1894  		}
  1895  		if data.bats[i].Length() == 0 {
  1896  			if block, size, err = writer.WriteSubBatch(containers.ToCNBatch(data.bats[i]), objectio.ConvertToSchemaType(uint16(i))); err != nil {
  1897  				return
  1898  			}
  1899  			blockLoc := BuildBlockLoaction(block.GetID(), uint64(offset), uint64(0))
  1900  			indexes[i] = append(indexes[i], blockIndexes{
  1901  				fileNum: fileNum,
  1902  				indexes: &blockLoc,
  1903  			})
  1904  			schemaTypes = append(schemaTypes, uint16(i))
  1905  			objectSize += size
  1906  		} else {
  1907  			split := containers.NewBatchSplitter(data.bats[i], blockRows)
  1908  			for {
  1909  				bat, err = split.Next()
  1910  				if err != nil {
  1911  					break
  1912  				}
  1913  				defer bat.Close()
  1914  				if block, size, err = writer.WriteSubBatch(containers.ToCNBatch(bat), objectio.ConvertToSchemaType(uint16(i))); err != nil {
  1915  					return
  1916  				}
  1917  				Endoffset := offset + bat.Length()
  1918  				blockLoc := BuildBlockLoaction(block.GetID(), uint64(offset), uint64(Endoffset))
  1919  				indexes[i] = append(indexes[i], blockIndexes{
  1920  					fileNum: fileNum,
  1921  					indexes: &blockLoc,
  1922  				})
  1923  				schemaTypes = append(schemaTypes, uint16(i))
  1924  				offset += bat.Length()
  1925  				objectSize += size
  1926  			}
  1927  		}
  1928  	}
  1929  	blks, _, err := writer.Sync(context.Background())
  1930  	if err != nil {
  1931  		return
  1932  	}
  1933  	checkpointFiles = append(checkpointFiles, name.String())
  1934  	schemas = append(schemas, schemaTypes)
  1935  	objectBlocks = append(objectBlocks, blks)
  1936  
  1937  	data.prepareTNMetaBatch(checkpointNames, objectBlocks, schemas)
  1938  
  1939  	for tid, mata := range data.meta {
  1940  		for i, table := range mata.tables {
  1941  			if table == nil || table.ClosedInterval.Start == table.ClosedInterval.End {
  1942  				continue
  1943  			}
  1944  
  1945  			if i > BlockDelete {
  1946  				if tid == pkgcatalog.MO_DATABASE_ID ||
  1947  					tid == pkgcatalog.MO_TABLES_ID ||
  1948  					tid == pkgcatalog.MO_COLUMNS_ID {
  1949  					break
  1950  				}
  1951  			}
  1952  			idx := switchCheckpointIdx(uint16(i), tid)
  1953  			for _, blockIdx := range indexes[idx] {
  1954  				block := blockIdx.indexes
  1955  				name = checkpointNames[blockIdx.fileNum]
  1956  				if table.End <= block.GetStartOffset() {
  1957  					break
  1958  				}
  1959  				if table.Start >= block.GetEndOffset() {
  1960  					continue
  1961  				}
  1962  				blks = objectBlocks[blockIdx.fileNum]
  1963  				//blockLoc1 := objectio.BuildLocation(name, blks[block.GetID()].GetExtent(), 0, block.GetID())
  1964  				//logutil.Infof("write block %v to %d-%d, table is %d-%d", blockLoc1.String(), block.GetStartOffset(), block.GetEndOffset(), table.Start, table.End)
  1965  				if table.Uint64Contains(block.GetStartOffset(), block.GetEndOffset()) {
  1966  					blockLoc := BuildBlockLoactionWithLocation(
  1967  						name, blks[block.GetID()].GetExtent(), 0, block.GetID(),
  1968  						0, block.GetEndOffset()-block.GetStartOffset())
  1969  					table.locations.Append(blockLoc)
  1970  				} else if block.Contains(table.ClosedInterval) {
  1971  					blockLoc := BuildBlockLoactionWithLocation(
  1972  						name, blks[block.GetID()].GetExtent(), 0, block.GetID(),
  1973  						table.Start-block.GetStartOffset(), table.End-block.GetStartOffset())
  1974  					table.locations.Append(blockLoc)
  1975  				} else if table.Start <= block.GetEndOffset() && table.Start >= block.GetStartOffset() {
  1976  					blockLoc := BuildBlockLoactionWithLocation(
  1977  						name, blks[block.GetID()].GetExtent(), 0, block.GetID(),
  1978  						table.Start-block.GetStartOffset(), block.GetEndOffset()-block.GetStartOffset())
  1979  					table.locations.Append(blockLoc)
  1980  				} else if table.End <= block.GetEndOffset() && table.End >= block.GetStartOffset() {
  1981  					blockLoc := BuildBlockLoactionWithLocation(
  1982  						name, blks[block.GetID()].GetExtent(), 0, block.GetID(),
  1983  						0, table.End-block.GetStartOffset())
  1984  					table.locations.Append(blockLoc)
  1985  				}
  1986  			}
  1987  		}
  1988  	}
  1989  
  1990  	data.meta[0] = NewCheckpointMeta()
  1991  	data.meta[0].tables[0] = NewTableMeta()
  1992  	for num, fileName := range checkpointNames {
  1993  		loc := objectBlocks[num][0]
  1994  		blockLoc := BuildBlockLoactionWithLocation(
  1995  			fileName, loc.GetExtent(), 0, loc.GetID(),
  1996  			0, 0)
  1997  		data.meta[0].tables[0].locations.Append(blockLoc)
  1998  	}
  1999  	data.prepareMeta()
  2000  	if err != nil {
  2001  		return
  2002  	}
  2003  
  2004  	segmentid2 := objectio.NewSegmentid()
  2005  	name2 := objectio.BuildObjectName(segmentid2, 0)
  2006  	writer2, err := blockio.NewBlockWriterNew(fs, name2, 0, nil)
  2007  	if err != nil {
  2008  		return
  2009  	}
  2010  	if _, _, err = writer2.WriteSubBatch(
  2011  		containers.ToCNBatch(data.bats[MetaIDX]),
  2012  		objectio.ConvertToSchemaType(uint16(MetaIDX))); err != nil {
  2013  		return
  2014  	}
  2015  	if err != nil {
  2016  		return
  2017  	}
  2018  	if _, _, err = writer2.WriteSubBatch(
  2019  		containers.ToCNBatch(data.bats[TNMetaIDX]),
  2020  		objectio.ConvertToSchemaType(uint16(TNMetaIDX))); err != nil {
  2021  		return
  2022  	}
  2023  	if err != nil {
  2024  		return
  2025  	}
  2026  	blks2, _, err := writer2.Sync(context.Background())
  2027  	CNLocation = objectio.BuildLocation(name2, blks2[0].GetExtent(), 0, blks2[0].GetID())
  2028  	TNLocation = objectio.BuildLocation(name2, blks2[1].GetExtent(), 0, blks2[1].GetID())
  2029  	return
  2030  }
  2031  
  2032  func validateBeforeLoadBlkCol(version uint32, idxs []uint16, colNames []string) []uint16 {
  2033  	// in version 11, the storage usage ins/del was added into the ckp meta batch
  2034  	if version <= CheckpointVersion10 {
  2035  		if colNames[len(colNames)-1] == CheckpointMetaAttr_StorageUsageDelLocation {
  2036  			return idxs[0 : len(idxs)-2]
  2037  		}
  2038  	}
  2039  	return idxs
  2040  }
  2041  
  2042  func LoadBlkColumnsByMeta(
  2043  	version uint32,
  2044  	cxt context.Context,
  2045  	colTypes []types.Type,
  2046  	colNames []string,
  2047  	id uint16,
  2048  	reader *blockio.BlockReader,
  2049  	mp *mpool.MPool,
  2050  ) ([]*containers.Batch, error) {
  2051  	idxs := make([]uint16, len(colNames))
  2052  	for i := range colNames {
  2053  		idxs[i] = uint16(i)
  2054  	}
  2055  	var err error
  2056  	var ioResults []*batch.Batch
  2057  	var releases func()
  2058  	defer func() {
  2059  		if releases != nil {
  2060  			releases()
  2061  		}
  2062  	}()
  2063  	if version <= CheckpointVersion4 {
  2064  		ioResults = make([]*batch.Batch, 1)
  2065  		ioResults[0], releases, err = reader.LoadColumns(cxt, idxs, nil, id, nil)
  2066  	} else {
  2067  		idxs = validateBeforeLoadBlkCol(version, idxs, colNames)
  2068  		ioResults, releases, err = reader.LoadSubColumns(cxt, idxs, nil, id, nil)
  2069  	}
  2070  	if err != nil {
  2071  		return nil, err
  2072  	}
  2073  	bats := make([]*containers.Batch, 0)
  2074  	for _, ioResult := range ioResults {
  2075  		bat := containers.NewBatch()
  2076  		for i, idx := range idxs {
  2077  			pkgVec := ioResult.Vecs[i]
  2078  			var vec containers.Vector
  2079  			if pkgVec.Length() == 0 {
  2080  				vec = containers.MakeVector(colTypes[i], mp)
  2081  			} else {
  2082  				srcVec := containers.ToTNVector(pkgVec, mp)
  2083  				defer srcVec.Close()
  2084  				vec = srcVec.CloneWindow(0, srcVec.Length(), mp)
  2085  			}
  2086  			bat.AddVector(colNames[idx], vec)
  2087  			bat.Vecs[i] = vec
  2088  
  2089  		}
  2090  		bats = append(bats, bat)
  2091  	}
  2092  	return bats, nil
  2093  }
  2094  
  2095  func LoadCNSubBlkColumnsByMeta(
  2096  	version uint32,
  2097  	cxt context.Context,
  2098  	colTypes []types.Type,
  2099  	colNames []string,
  2100  	id uint16,
  2101  	reader *blockio.BlockReader,
  2102  	m *mpool.MPool,
  2103  ) ([]*batch.Batch, error) {
  2104  	idxs := make([]uint16, len(colNames))
  2105  	for i := range colNames {
  2106  		idxs[i] = uint16(i)
  2107  	}
  2108  	var err error
  2109  	var ioResults []*batch.Batch
  2110  	var release func()
  2111  	bats := make([]*batch.Batch, 0)
  2112  	defer func() {
  2113  		if release != nil {
  2114  			release()
  2115  		}
  2116  	}()
  2117  	if version <= CheckpointVersion4 {
  2118  		ioResults = make([]*batch.Batch, 1)
  2119  		ioResults[0], release, err = reader.LoadColumns(cxt, idxs, nil, id, nil)
  2120  	} else {
  2121  		idxs = validateBeforeLoadBlkCol(version, idxs, colNames)
  2122  		ioResults, release, err = reader.LoadSubColumns(cxt, idxs, nil, id, m)
  2123  	}
  2124  	if err != nil {
  2125  		return nil, err
  2126  	}
  2127  	for i := range ioResults {
  2128  		ioResults[i].Attrs = make([]string, len(colNames))
  2129  		copy(ioResults[i].Attrs, colNames)
  2130  		var bat *batch.Batch
  2131  		bat, err = ioResults[i].Dup(m)
  2132  		if err != nil {
  2133  			return nil, err
  2134  		}
  2135  		bats = append(bats, bat)
  2136  	}
  2137  	return bats, nil
  2138  }
  2139  
  2140  func LoadCNSubBlkColumnsByMetaWithId(
  2141  	cxt context.Context,
  2142  	colTypes []types.Type,
  2143  	colNames []string,
  2144  	dataType uint16,
  2145  	id uint16,
  2146  	version uint32,
  2147  	reader *blockio.BlockReader,
  2148  	m *mpool.MPool,
  2149  ) (bat *batch.Batch, err error) {
  2150  	idxs := make([]uint16, len(colNames))
  2151  	for i := range colNames {
  2152  		idxs[i] = uint16(i)
  2153  	}
  2154  	var ioResult *batch.Batch
  2155  	var release func()
  2156  	defer func() {
  2157  		if release != nil {
  2158  			release()
  2159  		}
  2160  	}()
  2161  	if version <= CheckpointVersion3 {
  2162  		ioResult, release, err = reader.LoadColumns(cxt, idxs, nil, id, nil)
  2163  	} else {
  2164  		idxs = validateBeforeLoadBlkCol(version, idxs, colNames)
  2165  		ioResult, release, err = reader.LoadOneSubColumns(cxt, idxs, nil, dataType, id, m)
  2166  	}
  2167  	if err != nil {
  2168  		return nil, err
  2169  	}
  2170  	ioResult.Attrs = make([]string, len(colNames))
  2171  	copy(ioResult.Attrs, colNames)
  2172  	return ioResult.Dup(m)
  2173  }
  2174  func (data *CheckpointData) ReadTNMetaBatch(
  2175  	ctx context.Context,
  2176  	version uint32,
  2177  	location objectio.Location,
  2178  	reader *blockio.BlockReader,
  2179  ) (err error) {
  2180  	if data.bats[TNMetaIDX].Length() == 0 {
  2181  		if version < CheckpointVersion5 {
  2182  			for i := 2; i < MetaMaxIdx; i++ {
  2183  				location := objectio.BuildLocation(location.Name(), location.Extent(), 0, uint16(i))
  2184  				data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_BlockLocation).Append([]byte(location), false)
  2185  				data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_SchemaType).Append(uint16(i), false)
  2186  			}
  2187  		} else {
  2188  			var bats []*containers.Batch
  2189  			item := checkpointDataReferVersions[version][TNMetaIDX]
  2190  			bats, err = LoadBlkColumnsByMeta(version, ctx, item.types, item.attrs, TNMetaIDX, reader, data.allocator)
  2191  			if err != nil {
  2192  				return
  2193  			}
  2194  			data.bats[TNMetaIDX] = bats[0]
  2195  		}
  2196  	}
  2197  	return
  2198  }
  2199  
  2200  func (data *CheckpointData) PrefetchMeta(
  2201  	ctx context.Context,
  2202  	version uint32,
  2203  	service fileservice.FileService,
  2204  	key objectio.Location) (err error) {
  2205  	if version < CheckpointVersion4 {
  2206  		return
  2207  	}
  2208  	var pref blockio.PrefetchParams
  2209  	pref, err = blockio.BuildPrefetchParams(service, key)
  2210  	if err != nil {
  2211  		return
  2212  	}
  2213  	meteIdxSchema := checkpointDataReferVersions[version][MetaIDX]
  2214  	tnMeteIdxSchema := checkpointDataReferVersions[version][TNMetaIDX]
  2215  	idxes := make([]uint16, 0)
  2216  	tnIdxes := make([]uint16, 0)
  2217  	for attr := range meteIdxSchema.attrs {
  2218  		idxes = append(idxes, uint16(attr))
  2219  	}
  2220  	for attr := range tnMeteIdxSchema.attrs {
  2221  		tnIdxes = append(tnIdxes, uint16(attr))
  2222  	}
  2223  	pref.AddBlockWithType(idxes, []uint16{0}, uint16(objectio.ConvertToSchemaType(MetaIDX)))
  2224  	pref.AddBlockWithType(tnIdxes, []uint16{1}, uint16(objectio.ConvertToSchemaType(TNMetaIDX)))
  2225  	return blockio.PrefetchWithMerged(pref)
  2226  }
  2227  
  2228  type blockIdx struct {
  2229  	location objectio.Location
  2230  	dataType uint16
  2231  }
  2232  
  2233  func (data *CheckpointData) PrefetchFrom(
  2234  	ctx context.Context,
  2235  	version uint32,
  2236  	service fileservice.FileService,
  2237  	key objectio.Location) (err error) {
  2238  	if version < CheckpointVersion4 {
  2239  		return prefetchCheckpointData(ctx, version, service, key)
  2240  	}
  2241  	blocks := vector.MustBytesCol(data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_BlockLocation).GetDownstreamVector())
  2242  	dataType := vector.MustFixedCol[uint16](data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_SchemaType).GetDownstreamVector())
  2243  	var pref blockio.PrefetchParams
  2244  	locations := make(map[string][]blockIdx)
  2245  	checkpointSize := uint64(0)
  2246  	for i := 0; i < len(blocks); i++ {
  2247  		location := objectio.Location(blocks[i])
  2248  		if location.IsEmpty() {
  2249  			continue
  2250  		}
  2251  		name := location.Name()
  2252  		if locations[name.String()] == nil {
  2253  			locations[name.String()] = make([]blockIdx, 0)
  2254  		}
  2255  		locations[name.String()] = append(locations[name.String()], blockIdx{location: location, dataType: dataType[i]})
  2256  	}
  2257  	for _, blockIdxes := range locations {
  2258  		pref, err = blockio.BuildPrefetchParams(service, blockIdxes[0].location)
  2259  		if err != nil {
  2260  			return
  2261  		}
  2262  		checkpointSize += uint64(blockIdxes[0].location.Extent().End())
  2263  		logutil.Info("prefetch-read-checkpoint", common.OperationField("prefetch read"),
  2264  			common.OperandField("checkpoint"),
  2265  			common.AnyField("location", blockIdxes[0].location.String()),
  2266  			common.AnyField("size", checkpointSize))
  2267  		for _, idx := range blockIdxes {
  2268  			schema := checkpointDataReferVersions[version][idx.dataType]
  2269  			idxes := make([]uint16, len(schema.attrs))
  2270  			for attr := range schema.attrs {
  2271  				idxes[attr] = uint16(attr)
  2272  			}
  2273  			pref.AddBlockWithType(idxes, []uint16{idx.location.ID()}, uint16(objectio.ConvertToSchemaType(idx.dataType)))
  2274  		}
  2275  		err = blockio.PrefetchWithMerged(pref)
  2276  		if err != nil {
  2277  			logutil.Warnf("PrefetchFrom PrefetchWithMerged error %v", err)
  2278  		}
  2279  	}
  2280  	logutil.Info("prefetch-checkpoint",
  2281  		common.AnyField("size", checkpointSize),
  2282  		common.AnyField("count", len(locations)))
  2283  	return
  2284  }
  2285  
  2286  func prefetchCheckpointData(
  2287  	ctx context.Context,
  2288  	version uint32,
  2289  	service fileservice.FileService,
  2290  	key objectio.Location,
  2291  ) (err error) {
  2292  	var pref blockio.PrefetchParams
  2293  	pref, err = blockio.BuildPrefetchParams(service, key)
  2294  	if err != nil {
  2295  		return
  2296  	}
  2297  	for idx, item := range checkpointDataReferVersions[version] {
  2298  		idxes := make([]uint16, len(item.attrs))
  2299  		for i := range item.attrs {
  2300  			idxes[i] = uint16(i)
  2301  		}
  2302  		pref.AddBlock(idxes, []uint16{uint16(idx)})
  2303  	}
  2304  	return blockio.PrefetchWithMerged(pref)
  2305  }
  2306  
  2307  // TODO:
  2308  // There need a global io pool
  2309  func (data *CheckpointData) ReadFrom(
  2310  	ctx context.Context,
  2311  	version uint32,
  2312  	location objectio.Location,
  2313  	reader *blockio.BlockReader,
  2314  	fs fileservice.FileService,
  2315  ) (err error) {
  2316  	err = data.readMetaBatch(ctx, version, reader, data.allocator)
  2317  	if err != nil {
  2318  		return
  2319  	}
  2320  	if version <= CheckpointVersion4 {
  2321  		data.fillInMetaBatchWithLocation(location)
  2322  	}
  2323  	err = data.readAll(ctx, version, fs)
  2324  	if err != nil {
  2325  		return
  2326  	}
  2327  
  2328  	return
  2329  }
  2330  
  2331  func LoadCheckpointLocations(
  2332  	ctx context.Context,
  2333  	location objectio.Location,
  2334  	version uint32,
  2335  	fs fileservice.FileService,
  2336  ) (map[string]objectio.Location, error) {
  2337  	var err error
  2338  	data := NewCheckpointData(common.CheckpointAllocator)
  2339  	defer data.Close()
  2340  
  2341  	var reader *blockio.BlockReader
  2342  	if reader, err = blockio.NewObjectReader(fs, location); err != nil {
  2343  		return nil, err
  2344  	}
  2345  
  2346  	if err = data.readMetaBatch(ctx, version, reader, nil); err != nil {
  2347  		return nil, err
  2348  	}
  2349  
  2350  	data.replayMetaBatch(version)
  2351  	return data.locations, nil
  2352  }
  2353  
  2354  // LoadSpecifiedCkpBatch loads a specified checkpoint data batch
  2355  func LoadSpecifiedCkpBatch(
  2356  	ctx context.Context,
  2357  	location objectio.Location,
  2358  	version uint32,
  2359  	batchIdx uint16,
  2360  	fs fileservice.FileService,
  2361  ) (data *CheckpointData, err error) {
  2362  	data = NewCheckpointData(common.CheckpointAllocator)
  2363  	defer func() {
  2364  		if err != nil {
  2365  			data.Close()
  2366  			data = nil
  2367  		}
  2368  	}()
  2369  
  2370  	if batchIdx >= MaxIDX {
  2371  		err = moerr.NewInvalidArgNoCtx("out of bound batchIdx", batchIdx)
  2372  		return
  2373  	}
  2374  	var reader *blockio.BlockReader
  2375  	if reader, err = blockio.NewObjectReader(fs, location); err != nil {
  2376  		return
  2377  	}
  2378  
  2379  	if err = data.readMetaBatch(ctx, version, reader, nil); err != nil {
  2380  		return
  2381  	}
  2382  
  2383  	data.replayMetaBatch(version)
  2384  	for _, val := range data.locations {
  2385  		if reader, err = blockio.NewObjectReader(fs, val); err != nil {
  2386  			return
  2387  		}
  2388  		var bats []*containers.Batch
  2389  		item := checkpointDataReferVersions[version][batchIdx]
  2390  		if bats, err = LoadBlkColumnsByMeta(
  2391  			version, ctx, item.types, item.attrs, batchIdx, reader, data.allocator,
  2392  		); err != nil {
  2393  			return
  2394  		}
  2395  
  2396  		for i := range bats {
  2397  			if err = data.bats[batchIdx].Append(bats[i]); err != nil {
  2398  				return
  2399  			}
  2400  			defer bats[i].Close()
  2401  		}
  2402  	}
  2403  
  2404  	return data, nil
  2405  }
  2406  
  2407  func (data *CheckpointData) readMetaBatch(
  2408  	ctx context.Context,
  2409  	version uint32,
  2410  	reader *blockio.BlockReader,
  2411  	m *mpool.MPool,
  2412  ) (err error) {
  2413  	if data.bats[MetaIDX].Length() == 0 {
  2414  		var bats []*containers.Batch
  2415  		item := checkpointDataReferVersions[version][MetaIDX]
  2416  		if bats, err = LoadBlkColumnsByMeta(
  2417  			version, ctx, item.types, item.attrs, uint16(0), reader, data.allocator,
  2418  		); err != nil {
  2419  			return
  2420  		}
  2421  		data.bats[MetaIDX] = bats[0]
  2422  	}
  2423  	return
  2424  }
  2425  
  2426  func (data *CheckpointData) replayMetaBatch(version uint32) {
  2427  	bat := data.bats[MetaIDX]
  2428  	data.locations = make(map[string]objectio.Location)
  2429  	tidVec := vector.MustFixedCol[uint64](bat.GetVectorByName(SnapshotAttr_TID).GetDownstreamVector())
  2430  	insVec := bat.GetVectorByName(SnapshotMetaAttr_BlockInsertBatchLocation).GetDownstreamVector()
  2431  	delVec := bat.GetVectorByName(SnapshotMetaAttr_BlockCNInsertBatchLocation).GetDownstreamVector()
  2432  	delCNVec := bat.GetVectorByName(SnapshotMetaAttr_BlockDeleteBatchLocation).GetDownstreamVector()
  2433  	segVec := bat.GetVectorByName(SnapshotMetaAttr_SegDeleteBatchLocation).GetDownstreamVector()
  2434  
  2435  	var usageInsVec, usageDelVec *vector.Vector
  2436  	if version >= CheckpointVersion11 {
  2437  		usageInsVec = bat.GetVectorByName(CheckpointMetaAttr_StorageUsageInsLocation).GetDownstreamVector()
  2438  		usageDelVec = bat.GetVectorByName(CheckpointMetaAttr_StorageUsageDelLocation).GetDownstreamVector()
  2439  	}
  2440  
  2441  	for i := 0; i < len(tidVec); i++ {
  2442  		tid := tidVec[i]
  2443  		if tid == 0 {
  2444  			bl := BlockLocations(insVec.GetBytesAt(i))
  2445  			it := bl.MakeIterator()
  2446  			for it.HasNext() {
  2447  				block := it.Next()
  2448  				if !block.GetLocation().IsEmpty() {
  2449  					data.locations[block.GetLocation().Name().String()] = block.GetLocation()
  2450  				}
  2451  			}
  2452  			continue
  2453  		}
  2454  		insLocation := insVec.GetBytesAt(i)
  2455  		delLocation := delVec.GetBytesAt(i)
  2456  		delCNLocation := delCNVec.GetBytesAt(i)
  2457  		segLocation := segVec.GetBytesAt(i)
  2458  
  2459  		tmp := [][]byte{insLocation, delLocation, delCNLocation, segLocation}
  2460  		if usageInsVec != nil {
  2461  			tmp = append(tmp, usageInsVec.GetBytesAt(i))
  2462  			tmp = append(tmp, usageDelVec.GetBytesAt(i))
  2463  		}
  2464  
  2465  		tableMeta := NewCheckpointMeta()
  2466  		tableMeta.DecodeFromString(tmp)
  2467  		data.meta[tid] = tableMeta
  2468  	}
  2469  
  2470  	for _, meta := range data.meta {
  2471  		for _, table := range meta.tables {
  2472  			if table == nil {
  2473  				continue
  2474  			}
  2475  
  2476  			it := table.locations.MakeIterator()
  2477  			for it.HasNext() {
  2478  				block := it.Next()
  2479  				if !block.GetLocation().IsEmpty() {
  2480  					data.locations[block.GetLocation().Name().String()] = block.GetLocation()
  2481  				}
  2482  			}
  2483  		}
  2484  	}
  2485  }
  2486  
  2487  func (data *CheckpointData) readAll(
  2488  	ctx context.Context,
  2489  	version uint32,
  2490  	service fileservice.FileService,
  2491  ) (err error) {
  2492  	data.replayMetaBatch(version)
  2493  	checkpointDataSize := uint64(0)
  2494  	readDuration := time.Now()
  2495  	for _, val := range data.locations {
  2496  		var reader *blockio.BlockReader
  2497  		reader, err = blockio.NewObjectReader(service, val)
  2498  		if err != nil {
  2499  			return
  2500  		}
  2501  		var bats []*containers.Batch
  2502  		now := time.Now()
  2503  		for idx := range checkpointDataReferVersions[version] {
  2504  			if uint16(idx) == MetaIDX || uint16(idx) == TNMetaIDX {
  2505  				continue
  2506  			}
  2507  			item := checkpointDataReferVersions[version][idx]
  2508  
  2509  			if bats, err = LoadBlkColumnsByMeta(
  2510  				version, ctx, item.types, item.attrs, uint16(idx), reader, data.allocator,
  2511  			); err != nil {
  2512  				return
  2513  			}
  2514  			if version == CheckpointVersion1 {
  2515  				if uint16(idx) == TBLInsertIDX {
  2516  					for _, bat := range bats {
  2517  						length := bat.GetVectorByName(pkgcatalog.SystemRelAttr_Version).Length()
  2518  						vec := containers.MakeVector(types.T_uint32.ToType(), data.allocator)
  2519  						for i := 0; i < length; i++ {
  2520  							vec.Append(pkgcatalog.CatalogVersion_V1, false)
  2521  						}
  2522  						//Fixme: add vector to batch
  2523  						//bat.AddVector(pkgcatalog.SystemRelAttr_CatalogVersion, vec)
  2524  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemDBAttr_ID)
  2525  						bat.Vecs = append(bat.Vecs, vec)
  2526  					}
  2527  				}
  2528  			}
  2529  			if version <= CheckpointVersion2 {
  2530  				if uint16(idx) == DBDeleteIDX {
  2531  					for _, bat := range bats {
  2532  						rowIDVec := bat.GetVectorByName(catalog.AttrRowID)
  2533  						length := rowIDVec.Length()
  2534  						pkVec := containers.MakeVector(types.T_uint64.ToType(), data.allocator)
  2535  						for i := 0; i < length; i++ {
  2536  							pkVec.Append(objectio.HackRowidToU64(rowIDVec.Get(i).(types.Rowid)), false)
  2537  						}
  2538  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemDBAttr_ID)
  2539  						bat.Vecs = append(bat.Vecs, pkVec)
  2540  					}
  2541  				}
  2542  
  2543  				if uint16(idx) == TBLDeleteIDX {
  2544  					for _, bat := range bats {
  2545  						rowIDVec := bat.GetVectorByName(catalog.AttrRowID)
  2546  						length := rowIDVec.Length()
  2547  						pkVec2 := containers.MakeVector(types.T_uint64.ToType(), data.allocator)
  2548  						for i := 0; i < length; i++ {
  2549  							pkVec2.Append(objectio.HackRowidToU64(rowIDVec.Get(i).(types.Rowid)), false)
  2550  							if err != nil {
  2551  								return err
  2552  							}
  2553  						}
  2554  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemRelAttr_ID)
  2555  						bat.Vecs = append(bat.Vecs, pkVec2)
  2556  					}
  2557  				}
  2558  
  2559  				if uint16(idx) == TBLColDeleteIDX {
  2560  					for _, bat := range bats {
  2561  						rowIDVec := bat.GetVectorByName(catalog.AttrRowID)
  2562  						length := rowIDVec.Length()
  2563  						pkVec2 := containers.MakeVector(types.T_varchar.ToType(), data.allocator)
  2564  						for i := 0; i < length; i++ {
  2565  							pkVec2.Append(nil, true)
  2566  							if err != nil {
  2567  								return err
  2568  							}
  2569  						}
  2570  						bat.Attrs = append(bat.Attrs, pkgcatalog.SystemColAttr_UniqName)
  2571  						bat.Vecs = append(bat.Vecs, pkVec2)
  2572  					}
  2573  				}
  2574  			}
  2575  			if version <= CheckpointVersion3 {
  2576  				if uint16(idx) == TBLColInsertIDX {
  2577  					for _, bat := range bats {
  2578  						length := bat.GetVectorByName(catalog.AttrRowID).Length()
  2579  						vec := containers.MakeVector(types.New(types.T_varchar, types.MaxVarcharLen, 0), data.allocator)
  2580  						for i := 0; i < length; i++ {
  2581  							vec.Append([]byte(""), false)
  2582  						}
  2583  						bat.AddVector(pkgcatalog.SystemColAttr_EnumValues, vec)
  2584  					}
  2585  				}
  2586  			}
  2587  
  2588  			if version <= CheckpointVersion5 {
  2589  				if uint16(idx) == BLKTNMetaInsertIDX {
  2590  					for _, bat := range bats {
  2591  						committs := bat.GetVectorByName(pkgcatalog.BlockMeta_CommitTs)
  2592  						bat.AddVector(pkgcatalog.BlockMeta_MemTruncPoint, committs)
  2593  					}
  2594  				}
  2595  
  2596  				if uint16(idx) == BLKMetaInsertIDX {
  2597  					for _, bat := range bats {
  2598  						committs := bat.GetVectorByName(pkgcatalog.BlockMeta_CommitTs)
  2599  						bat.AddVector(pkgcatalog.BlockMeta_MemTruncPoint, committs)
  2600  					}
  2601  				}
  2602  
  2603  				if uint16(idx) == BLKCNMetaInsertIDX {
  2604  					for _, bat := range bats {
  2605  						committs := bat.GetVectorByName(pkgcatalog.BlockMeta_CommitTs)
  2606  						bat.AddVector(pkgcatalog.BlockMeta_MemTruncPoint, committs)
  2607  					}
  2608  				}
  2609  			}
  2610  			for i := range bats {
  2611  				data.bats[idx].Append(bats[i])
  2612  				bats[i].Close()
  2613  			}
  2614  		}
  2615  		logutil.Info("read-checkpoint", common.OperationField("read"),
  2616  			common.OperandField("checkpoint"),
  2617  			common.AnyField("location", val.String()),
  2618  			common.AnyField("size", val.Extent().End()),
  2619  			common.AnyField("read cost", time.Since(now)))
  2620  		checkpointDataSize += uint64(val.Extent().End())
  2621  	}
  2622  	logutil.Info("read-all", common.OperationField("read"),
  2623  		common.OperandField("checkpoint"),
  2624  		common.AnyField("size", checkpointDataSize),
  2625  		common.AnyField("duration", time.Since(readDuration)))
  2626  	return
  2627  }
  2628  
  2629  func (data *CheckpointData) ExportStats(prefix string) []zap.Field {
  2630  	fields := make([]zap.Field, 0, len(data.bats)+2)
  2631  	totalSize := 0
  2632  	totalRow := 0
  2633  	for idx := range data.bats {
  2634  		if data.bats[idx] == nil || data.bats[idx].Length() == 0 {
  2635  			continue
  2636  		}
  2637  		size := data.bats[idx].Allocated()
  2638  		rows := data.bats[idx].Length()
  2639  		totalSize += size
  2640  		totalRow += rows
  2641  		fields = append(fields, zap.Int(fmt.Sprintf("%s%s-Size", prefix, IDXString(uint16(idx))), size))
  2642  		fields = append(fields, zap.Int(fmt.Sprintf("%s%s-Row", prefix, IDXString(uint16(idx))), rows))
  2643  	}
  2644  	fields = append(fields, zap.Int(fmt.Sprintf("%stotalSize", prefix), totalSize))
  2645  	fields = append(fields, zap.Int(fmt.Sprintf("%stotalRow", prefix), totalRow))
  2646  	return fields
  2647  }
  2648  
  2649  func (data *CheckpointData) Close() {
  2650  	for idx := range data.bats {
  2651  		if data.bats[idx] != nil {
  2652  			data.bats[idx].Close()
  2653  			data.bats[idx] = nil
  2654  		}
  2655  	}
  2656  	data.allocator = nil
  2657  }
  2658  
  2659  func (data *CheckpointData) CloseWhenLoadFromCache(version uint32) {
  2660  	if version == CheckpointVersion1 {
  2661  		bat := data.bats[TBLInsertIDX]
  2662  		if bat == nil {
  2663  			return
  2664  		}
  2665  		vec := data.bats[TBLInsertIDX].GetVectorByName(pkgcatalog.SystemRelAttr_CatalogVersion)
  2666  		vec.Close()
  2667  	}
  2668  	if version <= CheckpointVersion2 {
  2669  		bat := data.bats[DBDeleteIDX]
  2670  		if bat == nil {
  2671  			return
  2672  		}
  2673  		vec := data.bats[DBDeleteIDX].GetVectorByName(pkgcatalog.SystemDBAttr_ID)
  2674  		vec.Close()
  2675  		bat = data.bats[TBLDeleteIDX]
  2676  		if bat == nil {
  2677  			return
  2678  		}
  2679  		vec = data.bats[TBLDeleteIDX].GetVectorByName(pkgcatalog.SystemRelAttr_ID)
  2680  		vec.Close()
  2681  	}
  2682  	if version <= CheckpointVersion3 {
  2683  		bat := data.bats[TBLColInsertIDX]
  2684  		if bat == nil {
  2685  			return
  2686  		}
  2687  		vec := data.bats[TBLColInsertIDX].GetVectorByName(pkgcatalog.SystemColAttr_EnumValues)
  2688  		vec.Close()
  2689  	}
  2690  	if version < CheckpointVersion5 {
  2691  		bat := data.bats[MetaIDX]
  2692  		if bat == nil {
  2693  			return
  2694  		}
  2695  		vec := data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockInsertBatchLocation)
  2696  		vec.Close()
  2697  		vec = data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockCNInsertBatchLocation)
  2698  		vec.Close()
  2699  		vec = data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_BlockDeleteBatchLocation)
  2700  		vec.Close()
  2701  		vec = data.bats[MetaIDX].GetVectorByName(SnapshotMetaAttr_SegDeleteBatchLocation)
  2702  		vec.Close()
  2703  
  2704  		bat = data.bats[TNMetaIDX]
  2705  		if bat == nil {
  2706  			return
  2707  		}
  2708  		vec = data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_BlockLocation)
  2709  		vec.Close()
  2710  		vec = data.bats[TNMetaIDX].GetVectorByName(CheckpointMetaAttr_SchemaType)
  2711  		vec.Close()
  2712  	}
  2713  }
  2714  func (data *CheckpointData) GetBatches() []*containers.Batch {
  2715  	return data.bats[:]
  2716  }
  2717  func (data *CheckpointData) GetDBBatchs() (
  2718  	*containers.Batch,
  2719  	*containers.Batch,
  2720  	*containers.Batch,
  2721  	*containers.Batch) {
  2722  	return data.bats[DBInsertIDX],
  2723  		data.bats[DBInsertTxnIDX],
  2724  		data.bats[DBDeleteIDX],
  2725  		data.bats[DBDeleteTxnIDX]
  2726  }
  2727  func (data *CheckpointData) GetTblBatchs() (
  2728  	*containers.Batch,
  2729  	*containers.Batch,
  2730  	*containers.Batch,
  2731  	*containers.Batch,
  2732  	*containers.Batch) {
  2733  	return data.bats[TBLInsertIDX],
  2734  		data.bats[TBLInsertTxnIDX],
  2735  		data.bats[TBLColInsertIDX],
  2736  		data.bats[TBLDeleteIDX],
  2737  		data.bats[TBLDeleteTxnIDX]
  2738  }
  2739  func (data *CheckpointData) GetTNObjectBatchs() *containers.Batch {
  2740  	return data.bats[TNObjectInfoIDX]
  2741  }
  2742  func (data *CheckpointData) GetObjectBatchs() *containers.Batch {
  2743  	return data.bats[ObjectInfoIDX]
  2744  }
  2745  func (data *CheckpointData) GetBlkBatchs() (
  2746  	*containers.Batch,
  2747  	*containers.Batch,
  2748  	*containers.Batch,
  2749  	*containers.Batch) {
  2750  	return data.bats[BLKMetaInsertIDX],
  2751  		data.bats[BLKMetaInsertTxnIDX],
  2752  		data.bats[BLKMetaDeleteIDX],
  2753  		data.bats[BLKMetaDeleteTxnIDX]
  2754  }
  2755  func (data *CheckpointData) GetTNBlkBatchs() (
  2756  	*containers.Batch,
  2757  	*containers.Batch,
  2758  	*containers.Batch,
  2759  	*containers.Batch) {
  2760  	return data.bats[BLKTNMetaInsertIDX],
  2761  		data.bats[BLKTNMetaInsertTxnIDX],
  2762  		data.bats[BLKTNMetaDeleteIDX],
  2763  		data.bats[BLKTNMetaDeleteTxnIDX]
  2764  }
  2765  func (collector *BaseCollector) LoadAndCollectObject(c *catalog.Catalog, visitObject func(*catalog.ObjectEntry) error) error {
  2766  	if collector.isPrefetch {
  2767  		collector.isPrefetch = false
  2768  	} else {
  2769  		return nil
  2770  	}
  2771  	collector.data.bats[ObjectInfoIDX] = makeRespBatchFromSchema(ObjectInfoSchema, common.CheckpointAllocator)
  2772  	collector.data.bats[TNObjectInfoIDX] = makeRespBatchFromSchema(ObjectInfoSchema, common.CheckpointAllocator)
  2773  	err := collector.loadObjectInfo()
  2774  	if err != nil {
  2775  		return err
  2776  	}
  2777  	p := &catalog.LoopProcessor{}
  2778  	p.ObjectFn = visitObject
  2779  	err = c.RecurLoop(p)
  2780  	if moerr.IsMoErrCode(err, moerr.OkStopCurrRecur) {
  2781  		err = nil
  2782  	}
  2783  	return err
  2784  }
  2785  func (collector *BaseCollector) VisitDB(entry *catalog.DBEntry) error {
  2786  	if shouldIgnoreDBInLogtail(entry.ID) {
  2787  		return nil
  2788  	}
  2789  	mvccNodes := entry.ClonePreparedInRange(collector.start, collector.end)
  2790  	delStart := collector.data.bats[DBDeleteIDX].GetVectorByName(catalog.AttrRowID).Length()
  2791  	insStart := collector.data.bats[DBInsertIDX].GetVectorByName(catalog.AttrRowID).Length()
  2792  	for _, node := range mvccNodes {
  2793  		if node.IsAborted() {
  2794  			continue
  2795  		}
  2796  		dbNode := node
  2797  		var created, dropped bool
  2798  		if dbNode.HasDropCommitted() {
  2799  			dropped = true
  2800  			if dbNode.CreatedAt.Equal(&dbNode.DeletedAt) {
  2801  				created = true
  2802  			}
  2803  		} else {
  2804  			created = true
  2805  		}
  2806  		if dropped {
  2807  			collector.Usage.Deletes = append(collector.Usage.Deletes, entry)
  2808  			// delScehma is empty, it will just fill rowid / commit ts
  2809  			catalogEntry2Batch(
  2810  				collector.data.bats[DBDeleteIDX],
  2811  				entry,
  2812  				node,
  2813  				DBDelSchema,
  2814  				txnimpl.FillDBRow,
  2815  				objectio.HackU64ToRowid(entry.GetID()),
  2816  				dbNode.GetEnd())
  2817  			dbNode.TxnMVCCNode.AppendTuple(collector.data.bats[DBDeleteTxnIDX])
  2818  			collector.data.bats[DBDeleteTxnIDX].GetVectorByName(SnapshotAttr_DBID).Append(entry.GetID(), false)
  2819  		}
  2820  		if created {
  2821  			catalogEntry2Batch(collector.data.bats[DBInsertIDX],
  2822  				entry,
  2823  				node,
  2824  				catalog.SystemDBSchema,
  2825  				txnimpl.FillDBRow,
  2826  				objectio.HackU64ToRowid(entry.GetID()),
  2827  				dbNode.GetEnd())
  2828  			dbNode.TxnMVCCNode.AppendTuple(collector.data.bats[DBInsertTxnIDX])
  2829  		}
  2830  	}
  2831  	delEnd := collector.data.bats[DBDeleteIDX].GetVectorByName(catalog.AttrRowID).Length()
  2832  	insEnd := collector.data.bats[DBInsertIDX].GetVectorByName(catalog.AttrRowID).Length()
  2833  	collector.data.updateMOCatalog(pkgcatalog.MO_DATABASE_ID, int32(insStart), int32(insEnd), int32(delStart), int32(delEnd))
  2834  	return nil
  2835  }
  2836  func (collector *GlobalCollector) isEntryDeletedBeforeThreshold(entry catalog.BaseEntry) bool {
  2837  	entry.RLock()
  2838  	defer entry.RUnlock()
  2839  	return entry.DeleteBeforeLocked(collector.versionThershold)
  2840  }
  2841  func (collector *GlobalCollector) VisitDB(entry *catalog.DBEntry) error {
  2842  	if collector.isEntryDeletedBeforeThreshold(entry.BaseEntryImpl) {
  2843  		collector.Usage.Deletes = append(collector.Usage.Deletes, entry)
  2844  		return nil
  2845  	}
  2846  
  2847  	currAccId := uint32(entry.GetTenantID())
  2848  	collector.Usage.ReservedAccIds[uint64(currAccId)] = struct{}{}
  2849  
  2850  	return collector.BaseCollector.VisitDB(entry)
  2851  }
  2852  
  2853  func (collector *BaseCollector) Allocator() *mpool.MPool { return collector.data.allocator }
  2854  func (collector *BaseCollector) VisitTable(entry *catalog.TableEntry) (err error) {
  2855  	if shouldIgnoreTblInLogtail(entry.ID) {
  2856  		return nil
  2857  	}
  2858  	mvccNodes := entry.ClonePreparedInRange(collector.start, collector.end)
  2859  	tableColDelBat := collector.data.bats[TBLColDeleteIDX]
  2860  	tableDelTxnBat := collector.data.bats[TBLDeleteTxnIDX]
  2861  	tableDelBat := collector.data.bats[TBLDeleteIDX]
  2862  	tableColInsBat := collector.data.bats[TBLColInsertIDX]
  2863  	tableInsBat := collector.data.bats[TBLInsertIDX]
  2864  	tableColInsTxnBat := collector.data.bats[TBLInsertTxnIDX]
  2865  	tblDelStart := collector.data.bats[TBLDeleteIDX].GetVectorByName(catalog.AttrRowID).Length()
  2866  	tblInsStart := collector.data.bats[TBLInsertIDX].GetVectorByName(catalog.AttrRowID).Length()
  2867  	colDelStart := collector.data.bats[TBLColDeleteIDX].GetVectorByName(catalog.AttrRowID).Length()
  2868  	colInsStart := collector.data.bats[TBLColInsertIDX].GetVectorByName(catalog.AttrRowID).Length()
  2869  	for _, node := range mvccNodes {
  2870  		if node.IsAborted() {
  2871  			continue
  2872  		}
  2873  		tblNode := node
  2874  		var created, dropped bool
  2875  		if tblNode.HasDropCommitted() {
  2876  			dropped = true
  2877  			if tblNode.CreatedAt.Equal(&tblNode.DeletedAt) {
  2878  				created = true
  2879  			}
  2880  		} else {
  2881  			created = true
  2882  		}
  2883  		if created {
  2884  			for _, syscol := range catalog.SystemColumnSchema.ColDefs {
  2885  				txnimpl.FillColumnRow(
  2886  					entry,
  2887  					tblNode,
  2888  					syscol.Name,
  2889  					tableColInsBat.GetVectorByName(syscol.Name),
  2890  				)
  2891  			}
  2892  			// send dropped column del
  2893  			for _, name := range tblNode.BaseNode.Schema.Extra.DroppedAttrs {
  2894  				tableColDelBat.GetVectorByName(catalog.AttrRowID).Append(objectio.HackBytes2Rowid([]byte(fmt.Sprintf("%d-%s", entry.GetID(), name))), false)
  2895  				tableColDelBat.GetVectorByName(catalog.AttrCommitTs).Append(tblNode.GetEnd(), false)
  2896  				tableColDelBat.GetVectorByName(pkgcatalog.SystemColAttr_UniqName).Append([]byte(fmt.Sprintf("%d-%s", entry.GetID(), name)), false)
  2897  			}
  2898  			rowidVec := tableColInsBat.GetVectorByName(catalog.AttrRowID)
  2899  			commitVec := tableColInsBat.GetVectorByName(catalog.AttrCommitTs)
  2900  			for _, usercol := range tblNode.BaseNode.Schema.ColDefs {
  2901  				rowidVec.Append(objectio.HackBytes2Rowid([]byte(fmt.Sprintf("%d-%s", entry.GetID(), usercol.Name))), false)
  2902  				commitVec.Append(tblNode.GetEnd(), false)
  2903  			}
  2904  
  2905  			tableColInsTxnBat.GetVectorByName(
  2906  				SnapshotAttr_BlockMaxRow).Append(entry.GetLastestSchemaLocked().BlockMaxRows, false)
  2907  			tableColInsTxnBat.GetVectorByName(
  2908  				SnapshotAttr_ObjectMaxBlock).Append(entry.GetLastestSchemaLocked().ObjectMaxBlocks, false)
  2909  			tableColInsTxnBat.GetVectorByName(
  2910  				SnapshotAttr_SchemaExtra).Append(tblNode.BaseNode.Schema.MustGetExtraBytes(), false)
  2911  
  2912  			catalogEntry2Batch(
  2913  				tableInsBat,
  2914  				entry,
  2915  				tblNode,
  2916  				catalog.SystemTableSchema,
  2917  				txnimpl.FillTableRow,
  2918  				objectio.HackU64ToRowid(entry.GetID()),
  2919  				tblNode.GetEnd(),
  2920  			)
  2921  
  2922  			tblNode.TxnMVCCNode.AppendTuple(tableColInsTxnBat)
  2923  		}
  2924  		if dropped {
  2925  			collector.Usage.Deletes = append(collector.Usage.Deletes, entry)
  2926  			tableDelTxnBat.GetVectorByName(
  2927  				SnapshotAttr_DBID).Append(entry.GetDB().GetID(), false)
  2928  			tableDelTxnBat.GetVectorByName(
  2929  				SnapshotAttr_TID).Append(entry.GetID(), false)
  2930  
  2931  			rowidVec := tableColDelBat.GetVectorByName(catalog.AttrRowID)
  2932  			commitVec := tableColDelBat.GetVectorByName(catalog.AttrCommitTs)
  2933  			pkVec := tableColDelBat.GetVectorByName(pkgcatalog.SystemColAttr_UniqName)
  2934  			for _, usercol := range tblNode.BaseNode.Schema.ColDefs {
  2935  				rowidVec.Append(objectio.HackBytes2Rowid([]byte(fmt.Sprintf("%d-%s", entry.GetID(), usercol.Name))), false)
  2936  				commitVec.Append(tblNode.GetEnd(), false)
  2937  				pkVec.Append([]byte(fmt.Sprintf("%d-%s", entry.GetID(), usercol.Name)), false)
  2938  			}
  2939  
  2940  			catalogEntry2Batch(
  2941  				tableDelBat,
  2942  				entry,
  2943  				tblNode,
  2944  				TblDelSchema,
  2945  				txnimpl.FillTableRow,
  2946  				objectio.HackU64ToRowid(entry.GetID()),
  2947  				tblNode.GetEnd(),
  2948  			)
  2949  			tblNode.TxnMVCCNode.AppendTuple(tableDelTxnBat)
  2950  		}
  2951  	}
  2952  	tblDelEnd := collector.data.bats[TBLDeleteIDX].GetVectorByName(catalog.AttrRowID).Length()
  2953  	tblInsEnd := collector.data.bats[TBLInsertIDX].GetVectorByName(catalog.AttrRowID).Length()
  2954  	colDelEnd := collector.data.bats[TBLColDeleteIDX].GetVectorByName(catalog.AttrRowID).Length()
  2955  	colInsEnd := collector.data.bats[TBLColInsertIDX].GetVectorByName(catalog.AttrRowID).Length()
  2956  	collector.data.updateMOCatalog(pkgcatalog.MO_TABLES_ID, int32(tblInsStart), int32(tblInsEnd), int32(tblDelStart), int32(tblDelEnd))
  2957  	collector.data.updateMOCatalog(pkgcatalog.MO_COLUMNS_ID, int32(colInsStart), int32(colInsEnd), int32(colDelStart), int32(colDelEnd))
  2958  	return nil
  2959  }
  2960  
  2961  func (collector *GlobalCollector) VisitTable(entry *catalog.TableEntry) error {
  2962  	if collector.isEntryDeletedBeforeThreshold(entry.BaseEntryImpl) {
  2963  		collector.Usage.Deletes = append(collector.Usage.Deletes, entry)
  2964  		return nil
  2965  	}
  2966  	if collector.isEntryDeletedBeforeThreshold(entry.GetDB().BaseEntryImpl) {
  2967  		return nil
  2968  	}
  2969  	return collector.BaseCollector.VisitTable(entry)
  2970  }
  2971  
  2972  func (collector *BaseCollector) visitObjectEntry(entry *catalog.ObjectEntry) error {
  2973  	mvccNodes := entry.ClonePreparedInRange(collector.start, collector.end)
  2974  	if len(mvccNodes) == 0 {
  2975  		return nil
  2976  	}
  2977  
  2978  	var needPrefetch bool
  2979  	if !collector.skipLoadObjectStats {
  2980  		needPrefetch = entry.NeedPrefetchObjectMetaForObjectInfo(mvccNodes)
  2981  	}
  2982  
  2983  	if collector.isPrefetch {
  2984  		if needPrefetch {
  2985  			collector.Objects = append(collector.Objects, entry)
  2986  		}
  2987  	} else {
  2988  		if needPrefetch {
  2989  			collector.isPrefetch = true
  2990  			logutil.Infof("checkpoint %v->%v, when try visit object, object %v need to load stats",
  2991  				collector.start.ToString(),
  2992  				collector.end.ToString(),
  2993  				entry.ID.String())
  2994  			if collector.data.bats[ObjectInfoIDX] != nil {
  2995  				collector.data.bats[ObjectInfoIDX].Close()
  2996  			}
  2997  			if collector.data.bats[TNObjectInfoIDX] != nil {
  2998  				collector.data.bats[TNObjectInfoIDX].Close()
  2999  			}
  3000  			collector.data.resetObjectMeta()
  3001  			if collector.Objects == nil {
  3002  				collector.Objects = make([]*catalog.ObjectEntry, 0)
  3003  			}
  3004  			collector.Objects = append(collector.Objects, entry)
  3005  		} else {
  3006  			entry.SetObjectStatsForPreviousNode(mvccNodes)
  3007  			err := collector.fillObjectInfoBatch(entry, mvccNodes)
  3008  			if err != nil {
  3009  				return err
  3010  			}
  3011  		}
  3012  	}
  3013  
  3014  	return nil
  3015  }
  3016  func (collector *BaseCollector) loadObjectInfo() error {
  3017  	logutil.Infof("checkpoint %v->%v, start to load object meta, total %d objects",
  3018  		collector.start.ToString(),
  3019  		collector.end.ToString(),
  3020  		len(collector.Objects))
  3021  	t0 := time.Now()
  3022  	batchCnt := 100
  3023  	i := 0
  3024  	for idx := 1; idx <= len(collector.Objects); idx++ {
  3025  
  3026  		obj := collector.Objects[idx-1]
  3027  		blockio.PrefetchMeta(obj.GetObjectData().GetFs().Service, obj.GetLocation())
  3028  
  3029  		for idx%batchCnt == 0 && i < idx {
  3030  			obj := collector.Objects[i]
  3031  			mvccNodes := obj.ClonePreparedInRange(collector.start, collector.end)
  3032  			for _, node := range mvccNodes {
  3033  				if node.BaseNode.IsEmpty() {
  3034  					stats, err := obj.LoadObjectInfoWithTxnTS(node.Start)
  3035  					if err != nil {
  3036  						return err
  3037  					}
  3038  					obj.Lock()
  3039  					obj.SearchNodeLocked(node).BaseNode.ObjectStats = stats
  3040  					obj.Unlock()
  3041  				}
  3042  
  3043  			}
  3044  			i++
  3045  		}
  3046  	}
  3047  	for ; i < len(collector.Objects); i++ {
  3048  		obj := collector.Objects[i]
  3049  		mvccNodes := obj.ClonePreparedInRange(collector.start, collector.end)
  3050  		for _, node := range mvccNodes {
  3051  			if node.BaseNode.IsEmpty() {
  3052  				stats, err := obj.LoadObjectInfoWithTxnTS(node.Start)
  3053  				if err != nil {
  3054  					return err
  3055  				}
  3056  				obj.Lock()
  3057  				obj.SearchNodeLocked(node).BaseNode.ObjectStats = stats
  3058  				obj.Unlock()
  3059  			}
  3060  		}
  3061  	}
  3062  	logutil.Infof("checkpoint %v->%v, load %d object meta takes %v",
  3063  		collector.start.ToString(),
  3064  		collector.end.ToString(),
  3065  		len(collector.Objects),
  3066  		time.Since(t0))
  3067  	return nil
  3068  }
  3069  func (collector *BaseCollector) fillObjectInfoBatch(entry *catalog.ObjectEntry, mvccNodes []*catalog.MVCCNode[*catalog.ObjectMVCCNode]) error {
  3070  	if len(mvccNodes) == 0 {
  3071  		return nil
  3072  	}
  3073  	delStart := collector.data.bats[ObjectInfoIDX].GetVectorByName(catalog.ObjectAttr_ObjectStats).Length()
  3074  
  3075  	for _, node := range mvccNodes {
  3076  		if node.IsAborted() {
  3077  			continue
  3078  		}
  3079  		if entry.IsAppendable() && node.BaseNode.IsEmpty() {
  3080  			visitObject(collector.data.bats[TNObjectInfoIDX], entry, node, false, types.TS{})
  3081  		} else {
  3082  			if entry.IsAppendable() && node.DeletedAt.IsEmpty() {
  3083  				panic(fmt.Sprintf("logic error, object %v", entry.ID.String()))
  3084  			}
  3085  			visitObject(collector.data.bats[ObjectInfoIDX], entry, node, false, types.TS{})
  3086  		}
  3087  		objNode := node
  3088  
  3089  		// collect usage info
  3090  		if objNode.HasDropCommitted() {
  3091  			// deleted and non-append, record into the usage del bat
  3092  			if !entry.IsAppendable() && objNode.IsCommitted() {
  3093  				collector.Usage.ObjDeletes = append(collector.Usage.ObjDeletes, entry)
  3094  			}
  3095  		} else {
  3096  			// create and non-append, record into the usage ins bat
  3097  			if !entry.IsAppendable() && objNode.IsCommitted() {
  3098  				collector.Usage.ObjInserts = append(collector.Usage.ObjInserts, entry)
  3099  			}
  3100  		}
  3101  
  3102  	}
  3103  	delEnd := collector.data.bats[ObjectInfoIDX].GetVectorByName(catalog.ObjectAttr_ObjectStats).Length()
  3104  	collector.data.UpdateSegMeta(entry.GetTable().ID, int32(delStart), int32(delEnd))
  3105  	return nil
  3106  }
  3107  
  3108  func (collector *BaseCollector) VisitObjForBackup(entry *catalog.ObjectEntry) (err error) {
  3109  	entry.RLock()
  3110  	createTS := entry.GetCreatedAtLocked()
  3111  	if createTS.Greater(&collector.start) {
  3112  		entry.RUnlock()
  3113  		return nil
  3114  	}
  3115  	entry.RUnlock()
  3116  	return collector.visitObjectEntry(entry)
  3117  }
  3118  
  3119  func (collector *BaseCollector) VisitObj(entry *catalog.ObjectEntry) (err error) {
  3120  	collector.visitObjectEntry(entry)
  3121  	return nil
  3122  }
  3123  
  3124  func (collector *GlobalCollector) VisitObj(entry *catalog.ObjectEntry) error {
  3125  	if collector.isEntryDeletedBeforeThreshold(entry.BaseEntryImpl) && !entry.InMemoryDeletesExisted() {
  3126  		return nil
  3127  	}
  3128  	if collector.isEntryDeletedBeforeThreshold(entry.GetTable().BaseEntryImpl) {
  3129  		return nil
  3130  	}
  3131  	if collector.isEntryDeletedBeforeThreshold(entry.GetTable().GetDB().BaseEntryImpl) {
  3132  		return nil
  3133  	}
  3134  	return collector.BaseCollector.VisitObj(entry)
  3135  }
  3136  
  3137  func (collector *BaseCollector) visitTombstone(entry data.Tombstone) {
  3138  	// If ctx is used when collect in memory deletes.
  3139  	_, start, end, err := entry.VisitDeletes(context.Background(), collector.start, collector.end, collector.data.bats[BLKMetaInsertIDX], collector.data.bats[BLKMetaInsertTxnIDX], true)
  3140  	if err != nil {
  3141  		panic(err)
  3142  	}
  3143  	collector.data.UpdateBlkMeta(entry.GetObject().(*catalog.ObjectEntry).GetTable().ID, int32(start), int32(end), 0, 0)
  3144  }
  3145  
  3146  // TODO
  3147  // func (collector *BaseCollector) VisitBlkForBackup(entry *catalog.BlockEntry) (err error) {
  3148  // 	entry.RLock()
  3149  // 	if entry.GetCreatedAtLocked().Greater(collector.start) {
  3150  // 		entry.RUnlock()
  3151  // 		return nil
  3152  // 	}
  3153  // 	entry.RUnlock()
  3154  // 	collector.visitBlockEntry(entry)
  3155  // 	return nil
  3156  // }
  3157  
  3158  func (collector *BaseCollector) VisitTombstone(entry data.Tombstone) (err error) {
  3159  	collector.visitTombstone(entry)
  3160  	return nil
  3161  }
  3162  
  3163  func (collector *GlobalCollector) VisitTombstone(entry data.Tombstone) error {
  3164  	obj := entry.GetObject().(*catalog.ObjectEntry)
  3165  	if collector.isEntryDeletedBeforeThreshold(obj.BaseEntryImpl) && !obj.InMemoryDeletesExisted() {
  3166  		return nil
  3167  	}
  3168  	if collector.isEntryDeletedBeforeThreshold(obj.GetTable().BaseEntryImpl) {
  3169  		return nil
  3170  	}
  3171  	if collector.isEntryDeletedBeforeThreshold(obj.GetTable().GetDB().BaseEntryImpl) {
  3172  		return nil
  3173  	}
  3174  	return collector.BaseCollector.VisitTombstone(entry)
  3175  }
  3176  
  3177  func (collector *BaseCollector) OrphanData() *CheckpointData {
  3178  	data := collector.data
  3179  	collector.data = nil
  3180  	return data
  3181  }
  3182  
  3183  func (collector *BaseCollector) Close() {
  3184  	if collector.data != nil {
  3185  		collector.data.Close()
  3186  	}
  3187  }