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

     1  // Copyright 2022 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package disttae
    16  
    17  import (
    18  	"context"
    19  	"fmt"
    20  	"sort"
    21  	"strconv"
    22  	"strings"
    23  	"time"
    24  	"unsafe"
    25  
    26  	"github.com/matrixorigin/matrixone/pkg/pb/timestamp"
    27  
    28  	"github.com/matrixorigin/matrixone/pkg/catalog"
    29  	"github.com/matrixorigin/matrixone/pkg/common/moerr"
    30  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    31  	"github.com/matrixorigin/matrixone/pkg/container/batch"
    32  	"github.com/matrixorigin/matrixone/pkg/container/types"
    33  	"github.com/matrixorigin/matrixone/pkg/container/vector"
    34  	"github.com/matrixorigin/matrixone/pkg/defines"
    35  	"github.com/matrixorigin/matrixone/pkg/fileservice"
    36  	"github.com/matrixorigin/matrixone/pkg/logutil"
    37  	"github.com/matrixorigin/matrixone/pkg/objectio"
    38  	"github.com/matrixorigin/matrixone/pkg/pb/api"
    39  	"github.com/matrixorigin/matrixone/pkg/pb/plan"
    40  	pb "github.com/matrixorigin/matrixone/pkg/pb/statsinfo"
    41  	"github.com/matrixorigin/matrixone/pkg/sql/colexec"
    42  	"github.com/matrixorigin/matrixone/pkg/sql/colexec/deletion"
    43  	plan2 "github.com/matrixorigin/matrixone/pkg/sql/plan"
    44  	"github.com/matrixorigin/matrixone/pkg/sql/plan/function"
    45  	"github.com/matrixorigin/matrixone/pkg/sql/plan/rule"
    46  	"github.com/matrixorigin/matrixone/pkg/sql/util"
    47  	"github.com/matrixorigin/matrixone/pkg/txn/client"
    48  	"github.com/matrixorigin/matrixone/pkg/txn/trace"
    49  	"github.com/matrixorigin/matrixone/pkg/util/errutil"
    50  	v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2"
    51  	"github.com/matrixorigin/matrixone/pkg/vm/engine"
    52  	"github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/cache"
    53  	"github.com/matrixorigin/matrixone/pkg/vm/engine/disttae/logtailreplay"
    54  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio"
    55  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    56  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/index"
    57  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/mergesort"
    58  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/options"
    59  	"github.com/matrixorigin/matrixone/pkg/vm/process"
    60  )
    61  
    62  const (
    63  	AllColumns = "*"
    64  )
    65  
    66  var _ engine.Relation = new(txnTable)
    67  
    68  func (tbl *txnTable) getEngine() engine.Engine {
    69  	return tbl.getTxn().engine
    70  }
    71  
    72  func (tbl *txnTable) getTxn() *Transaction {
    73  	return tbl.db.getTxn()
    74  }
    75  
    76  func (tbl *txnTable) Stats(ctx context.Context, sync bool) (*pb.StatsInfo, error) {
    77  	_, err := tbl.getPartitionState(ctx)
    78  	if err != nil {
    79  		logutil.Errorf("failed to get partition state of table %d: %v", tbl.tableId, err)
    80  		return nil, err
    81  	}
    82  	if !tbl.db.op.IsSnapOp() {
    83  		e := tbl.getEngine()
    84  		return e.Stats(ctx, pb.StatsInfoKey{
    85  			DatabaseID: tbl.db.databaseId,
    86  			TableID:    tbl.tableId,
    87  		}, sync), nil
    88  	}
    89  	info, err := tbl.stats(ctx)
    90  	if err != nil {
    91  		return nil, err
    92  	}
    93  	return info, nil
    94  }
    95  
    96  func (tbl *txnTable) stats(ctx context.Context) (*pb.StatsInfo, error) {
    97  	partitionState, err := tbl.getPartitionState(ctx)
    98  	if err != nil {
    99  		return nil, err
   100  	}
   101  	e := tbl.db.getEng()
   102  	var partitionsTableDef []*plan2.TableDef
   103  	var approxObjectNum int64
   104  	if tbl.partitioned > 0 {
   105  		partitionInfo := &plan2.PartitionByDef{}
   106  		if err := partitionInfo.UnMarshalPartitionInfo([]byte(tbl.partition)); err != nil {
   107  			logutil.Errorf("failed to unmarshal partition table: %v", err)
   108  			return nil, err
   109  		}
   110  		for _, partitionTableName := range partitionInfo.PartitionTableNames {
   111  			partitionTable, err := tbl.db.Relation(ctx, partitionTableName, nil)
   112  			if err != nil {
   113  				return nil, err
   114  			}
   115  			partitionsTableDef = append(partitionsTableDef, partitionTable.(*txnTable).tableDef)
   116  			var ps *logtailreplay.PartitionState
   117  			if !tbl.db.op.IsSnapOp() {
   118  				ps = e.getOrCreateLatestPart(tbl.db.databaseId, partitionTable.(*txnTable).tableId).Snapshot()
   119  			} else {
   120  				p, err := e.getOrCreateSnapPart(
   121  					ctx,
   122  					partitionTable.(*txnTable),
   123  					types.TimestampToTS(tbl.db.op.SnapshotTS()),
   124  				)
   125  				if err != nil {
   126  					return nil, err
   127  				}
   128  				ps = p.Snapshot()
   129  			}
   130  			approxObjectNum += int64(ps.ApproxObjectsNum())
   131  		}
   132  	} else {
   133  		approxObjectNum = int64(partitionState.ApproxObjectsNum())
   134  	}
   135  
   136  	if approxObjectNum == 0 {
   137  		// There are no objects flushed yet.
   138  		return nil, nil
   139  	}
   140  
   141  	stats := plan2.NewStatsInfo()
   142  	req := newUpdateStatsRequest(
   143  		tbl.tableDef,
   144  		partitionsTableDef,
   145  		partitionState,
   146  		e.fs,
   147  		types.TimestampToTS(tbl.db.op.SnapshotTS()),
   148  		approxObjectNum,
   149  		stats,
   150  	)
   151  	if err := UpdateStats(ctx, req); err != nil {
   152  		logutil.Errorf("failed to init stats info for table %d", tbl.tableId)
   153  		return nil, err
   154  	}
   155  	return stats, nil
   156  }
   157  
   158  func (tbl *txnTable) Rows(ctx context.Context) (uint64, error) {
   159  	var rows uint64
   160  	deletes := make(map[types.Rowid]struct{})
   161  	tbl.getTxn().forEachTableWrites(
   162  		tbl.db.databaseId,
   163  		tbl.tableId,
   164  		tbl.getTxn().GetSnapshotWriteOffset(),
   165  		func(entry Entry) {
   166  			if entry.typ == INSERT || entry.typ == INSERT_TXN {
   167  				rows = rows + uint64(entry.bat.RowCount())
   168  			} else {
   169  				if entry.bat.GetVector(0).GetType().Oid == types.T_Rowid {
   170  					/*
   171  						CASE:
   172  						create table t1(a int);
   173  						begin;
   174  						truncate t1; //txnDatabase.Truncate will DELETE mo_tables
   175  						show tables; // t1 must be shown
   176  					*/
   177  					if entry.databaseId == catalog.MO_CATALOG_ID &&
   178  						entry.tableId == catalog.MO_TABLES_ID &&
   179  						entry.truncate {
   180  						return
   181  					}
   182  					vs := vector.MustFixedCol[types.Rowid](entry.bat.GetVector(0))
   183  					for _, v := range vs {
   184  						deletes[v] = struct{}{}
   185  					}
   186  				}
   187  			}
   188  		})
   189  
   190  	ts := types.TimestampToTS(tbl.db.op.SnapshotTS())
   191  	partition, err := tbl.getPartitionState(ctx)
   192  	if err != nil {
   193  		return 0, err
   194  	}
   195  	iter := partition.NewRowsIter(ts, nil, false)
   196  	defer func() { _ = iter.Close() }()
   197  	for iter.Next() {
   198  		entry := iter.Entry()
   199  		if _, ok := deletes[entry.RowID]; ok {
   200  			continue
   201  		}
   202  		rows++
   203  	}
   204  	//s := e.Stats(ctx, pb.StatsInfoKey{
   205  	//	DatabaseID: tbl.db.databaseId,
   206  	//	TableID:    tbl.tableId,
   207  	//}, true)
   208  	s, _ := tbl.Stats(ctx, true)
   209  	if s == nil {
   210  		return rows, nil
   211  	}
   212  	return uint64(s.TableCnt) + rows, nil
   213  }
   214  
   215  func (tbl *txnTable) Size(ctx context.Context, columnName string) (uint64, error) {
   216  	ts := types.TimestampToTS(tbl.db.op.SnapshotTS())
   217  	part, err := tbl.getPartitionState(ctx)
   218  	if err != nil {
   219  		return 0, err
   220  	}
   221  
   222  	szInPart := uint64(0)
   223  	neededCols := make(map[string]*plan.ColDef)
   224  	cols := tbl.GetTableDef(ctx).Cols
   225  	found := false
   226  
   227  	for i := range cols {
   228  		if columnName == AllColumns || cols[i].Name == columnName {
   229  			neededCols[cols[i].Name] = cols[i]
   230  			found = true
   231  		}
   232  	}
   233  
   234  	if !found {
   235  		return 0, moerr.NewInvalidInput(ctx, "bad input column name %v", columnName)
   236  	}
   237  
   238  	deletes := make(map[types.Rowid]struct{})
   239  	tbl.getTxn().forEachTableWrites(
   240  		tbl.db.databaseId,
   241  		tbl.tableId,
   242  		tbl.getTxn().GetSnapshotWriteOffset(),
   243  		func(entry Entry) {
   244  			if entry.typ == INSERT || entry.typ == INSERT_TXN {
   245  				for i, s := range entry.bat.Attrs {
   246  					if _, ok := neededCols[s]; ok {
   247  						szInPart += uint64(entry.bat.Vecs[i].Size())
   248  					}
   249  				}
   250  			} else {
   251  				if entry.bat.GetVector(0).GetType().Oid == types.T_Rowid {
   252  					// CASE:
   253  					// create table t1(a int);
   254  					// begin;
   255  					// truncate t1; //txnDatabase.Truncate will DELETE mo_tables
   256  					// show tables; // t1 must be shown
   257  					if entry.databaseId == catalog.MO_CATALOG_ID &&
   258  						entry.tableId == catalog.MO_TABLES_ID &&
   259  						entry.truncate {
   260  						return
   261  					}
   262  					vs := vector.MustFixedCol[types.Rowid](entry.bat.GetVector(0))
   263  					for _, v := range vs {
   264  						deletes[v] = struct{}{}
   265  					}
   266  				}
   267  			}
   268  		})
   269  
   270  	// Different rows may belong to same batch. So we have
   271  	// to record the batch which we have already handled to avoid
   272  	// repetitive computation
   273  	handled := make(map[*batch.Batch]struct{})
   274  	// Calculate the in mem size
   275  	// TODO: It might includ some deleted row size
   276  	iter := part.NewRowsIter(ts, nil, false)
   277  	defer func() { _ = iter.Close() }()
   278  	for iter.Next() {
   279  		entry := iter.Entry()
   280  		if _, ok := deletes[entry.RowID]; ok {
   281  			continue
   282  		}
   283  		if _, ok := handled[entry.Batch]; ok {
   284  			continue
   285  		}
   286  		for i, s := range entry.Batch.Attrs {
   287  			if _, ok := neededCols[s]; ok {
   288  				szInPart += uint64(entry.Batch.Vecs[i].Size())
   289  			}
   290  		}
   291  		handled[entry.Batch] = struct{}{}
   292  	}
   293  
   294  	//s := e.Stats(ctx, pb.StatsInfoKey{
   295  	//	DatabaseID: tbl.db.databaseId,
   296  	//	TableID:    tbl.tableId,
   297  	//}, true)
   298  	s, _ := tbl.Stats(ctx, true)
   299  	if s == nil {
   300  		return szInPart, nil
   301  	}
   302  	if columnName == AllColumns {
   303  		var ret uint64
   304  		for _, z := range s.SizeMap {
   305  			ret += z
   306  		}
   307  		return ret + szInPart, nil
   308  	}
   309  	sz, ok := s.SizeMap[columnName]
   310  	if !ok {
   311  		return 0, moerr.NewInvalidInput(ctx, "bad input column name %v", columnName)
   312  	}
   313  	return sz + szInPart, nil
   314  }
   315  
   316  func ForeachVisibleDataObject(
   317  	state *logtailreplay.PartitionState,
   318  	ts types.TS,
   319  	fn func(obj logtailreplay.ObjectEntry) error,
   320  ) (err error) {
   321  	iter, err := state.NewObjectsIter(ts)
   322  	if err != nil {
   323  		return err
   324  	}
   325  	for iter.Next() {
   326  		entry := iter.Entry()
   327  		if err = fn(entry); err != nil {
   328  			break
   329  		}
   330  	}
   331  	iter.Close()
   332  	return
   333  }
   334  
   335  // not accurate!  only used by stats
   336  func (tbl *txnTable) ApproxObjectsNum(ctx context.Context) int {
   337  	part, err := tbl.getPartitionState(ctx)
   338  	if err != nil {
   339  		return 0
   340  	}
   341  	return part.ApproxObjectsNum()
   342  }
   343  
   344  func (tbl *txnTable) MaxAndMinValues(ctx context.Context) ([][2]any, []uint8, error) {
   345  	var (
   346  		err  error
   347  		part *logtailreplay.PartitionState
   348  	)
   349  	if part, err = tbl.getPartitionState(ctx); err != nil {
   350  		return nil, nil, err
   351  	}
   352  
   353  	var inited bool
   354  	cols := tbl.GetTableDef(ctx).GetCols()
   355  	dataLength := len(cols) - 1
   356  
   357  	tableVal := make([][2]any, dataLength)
   358  	tableTypes := make([]uint8, dataLength)
   359  	zms := make([]objectio.ZoneMap, dataLength)
   360  
   361  	var meta objectio.ObjectDataMeta
   362  	var objMeta objectio.ObjectMeta
   363  	fs, err := fileservice.Get[fileservice.FileService](
   364  		tbl.getTxn().proc.FileService,
   365  		defines.SharedFileServiceName)
   366  	if err != nil {
   367  		return nil, nil, err
   368  	}
   369  	onObjFn := func(obj logtailreplay.ObjectEntry) error {
   370  		var err error
   371  		location := obj.Location()
   372  		if objMeta, err = objectio.FastLoadObjectMeta(ctx, &location, false, fs); err != nil {
   373  			return err
   374  		}
   375  		meta = objMeta.MustDataMeta()
   376  		if inited {
   377  			for idx := range zms {
   378  				zm := meta.MustGetColumn(uint16(cols[idx].Seqnum)).ZoneMap()
   379  				if !zm.IsInited() {
   380  					continue
   381  				}
   382  				index.UpdateZM(zms[idx], zm.GetMaxBuf())
   383  				index.UpdateZM(zms[idx], zm.GetMinBuf())
   384  			}
   385  		} else {
   386  			for idx := range zms {
   387  				zms[idx] = meta.MustGetColumn(uint16(cols[idx].Seqnum)).ZoneMap()
   388  				tableTypes[idx] = uint8(cols[idx].Typ.Id)
   389  			}
   390  			inited = true
   391  		}
   392  
   393  		return nil
   394  	}
   395  
   396  	if err = ForeachVisibleDataObject(
   397  		part,
   398  		types.TimestampToTS(tbl.db.op.SnapshotTS()),
   399  		onObjFn); err != nil {
   400  		return nil, nil, err
   401  	}
   402  
   403  	if !inited {
   404  		return nil, nil, moerr.NewInvalidInputNoCtx("table meta is nil")
   405  	}
   406  
   407  	for idx, zm := range zms {
   408  		tableVal[idx] = [2]any{zm.GetMin(), zm.GetMax()}
   409  	}
   410  
   411  	return tableVal, tableTypes, nil
   412  }
   413  
   414  func (tbl *txnTable) GetColumMetadataScanInfo(ctx context.Context, name string) ([]*plan.MetadataScanInfo, error) {
   415  	state, err := tbl.getPartitionState(ctx)
   416  	if err != nil {
   417  		return nil, err
   418  	}
   419  
   420  	cols := tbl.GetTableDef(ctx).GetCols()
   421  	found := false
   422  	n := 0
   423  	for _, c := range cols {
   424  		// TODO: We can keep hidden column but need a better way to know
   425  		// whether it has the colmeta or not
   426  		if !c.Hidden && (c.Name == name || name == AllColumns) {
   427  			n++
   428  			found = true
   429  		}
   430  	}
   431  	if !found {
   432  		return nil, moerr.NewInvalidInput(ctx, "bad input column name %v", name)
   433  	}
   434  
   435  	needCols := make([]*plan.ColDef, 0, n)
   436  	for _, c := range cols {
   437  		if !c.Hidden && (c.Name == name || name == AllColumns) {
   438  			needCols = append(needCols, c)
   439  		}
   440  	}
   441  
   442  	fs, err := fileservice.Get[fileservice.FileService](
   443  		tbl.getTxn().proc.FileService,
   444  		defines.SharedFileServiceName)
   445  	if err != nil {
   446  		return nil, err
   447  	}
   448  	infoList := make([]*plan.MetadataScanInfo, 0, state.ApproxObjectsNum())
   449  	onObjFn := func(obj logtailreplay.ObjectEntry) error {
   450  		createTs, err := obj.CreateTime.Marshal()
   451  		if err != nil {
   452  			return err
   453  		}
   454  		deleteTs, err := obj.DeleteTime.Marshal()
   455  		if err != nil {
   456  			return err
   457  		}
   458  
   459  		location := obj.Location()
   460  		objName := location.Name().String()
   461  
   462  		if name == AllColumns && obj.StatsValid() {
   463  			// no need to load object meta
   464  			for _, col := range needCols {
   465  				infoList = append(infoList, &plan.MetadataScanInfo{
   466  					ColName:    col.Name,
   467  					IsHidden:   col.Hidden,
   468  					ObjectName: objName,
   469  					ObjLoc:     location,
   470  					CreateTs:   createTs,
   471  					DeleteTs:   deleteTs,
   472  					RowCnt:     int64(obj.Rows()),
   473  					ZoneMap:    objectio.EmptyZm[:],
   474  				})
   475  			}
   476  			return nil
   477  		}
   478  
   479  		objMeta, err := objectio.FastLoadObjectMeta(ctx, &location, false, fs)
   480  		if err != nil {
   481  			return err
   482  		}
   483  		meta := objMeta.MustDataMeta()
   484  		rowCnt := int64(meta.BlockHeader().Rows())
   485  
   486  		for _, col := range needCols {
   487  			colMeta := meta.MustGetColumn(uint16(col.Seqnum))
   488  			infoList = append(infoList, &plan.MetadataScanInfo{
   489  				ColName:      col.Name,
   490  				IsHidden:     col.Hidden,
   491  				ObjectName:   objName,
   492  				ObjLoc:       location,
   493  				CreateTs:     createTs,
   494  				DeleteTs:     deleteTs,
   495  				RowCnt:       rowCnt,
   496  				NullCnt:      int64(colMeta.NullCnt()),
   497  				CompressSize: int64(colMeta.Location().Length()),
   498  				OriginSize:   int64(colMeta.Location().OriginSize()),
   499  				ZoneMap:      colMeta.ZoneMap(),
   500  			})
   501  		}
   502  		return nil
   503  	}
   504  
   505  	if err = ForeachVisibleDataObject(state, types.TimestampToTS(tbl.db.op.SnapshotTS()), onObjFn); err != nil {
   506  		return nil, err
   507  	}
   508  
   509  	var logStr string
   510  	for i, col := range needCols {
   511  		if i > 0 {
   512  			logStr += ", "
   513  		}
   514  		logStr += col.GetName()
   515  	}
   516  	logutil.Infof("cols in GetColumMetadataScanInfo: %s, result len: %d", logStr, len(infoList))
   517  
   518  	return infoList, nil
   519  }
   520  
   521  func (tbl *txnTable) GetDirtyPersistedBlks(state *logtailreplay.PartitionState) []types.Blockid {
   522  	tbl.getTxn().blockId_tn_delete_metaLoc_batch.RLock()
   523  	defer tbl.getTxn().blockId_tn_delete_metaLoc_batch.RUnlock()
   524  
   525  	dirtyBlks := make([]types.Blockid, 0)
   526  	for blk := range tbl.getTxn().blockId_tn_delete_metaLoc_batch.data {
   527  		if !state.BlockPersisted(blk) {
   528  			continue
   529  		}
   530  		dirtyBlks = append(dirtyBlks, blk)
   531  	}
   532  	return dirtyBlks
   533  }
   534  
   535  func (tbl *txnTable) LoadDeletesForBlock(bid types.Blockid, offsets *[]int64) (err error) {
   536  	tbl.getTxn().blockId_tn_delete_metaLoc_batch.RLock()
   537  	defer tbl.getTxn().blockId_tn_delete_metaLoc_batch.RUnlock()
   538  
   539  	bats, ok := tbl.getTxn().blockId_tn_delete_metaLoc_batch.data[bid]
   540  	if !ok {
   541  		return nil
   542  	}
   543  	for _, bat := range bats {
   544  		vs := vector.MustStrCol(bat.GetVector(0))
   545  		for _, deltaLoc := range vs {
   546  			location, err := blockio.EncodeLocationFromString(deltaLoc)
   547  			if err != nil {
   548  				return err
   549  			}
   550  			rowIdBat, release, err := blockio.LoadTombstoneColumns(
   551  				tbl.getTxn().proc.Ctx,
   552  				[]uint16{0},
   553  				nil,
   554  				tbl.getTxn().engine.fs,
   555  				location,
   556  				tbl.getTxn().proc.GetMPool())
   557  			if err != nil {
   558  				return err
   559  			}
   560  			defer release()
   561  			rowIds := vector.MustFixedCol[types.Rowid](rowIdBat.GetVector(0))
   562  			for _, rowId := range rowIds {
   563  				_, offset := rowId.Decode()
   564  				*offsets = append(*offsets, int64(offset))
   565  			}
   566  		}
   567  	}
   568  	return nil
   569  }
   570  
   571  // LoadDeletesForMemBlocksIn loads deletes for memory blocks whose data resides in PartitionState.rows
   572  func (tbl *txnTable) LoadDeletesForMemBlocksIn(
   573  	state *logtailreplay.PartitionState,
   574  	deletesRowId map[types.Rowid]uint8) error {
   575  
   576  	tbl.getTxn().blockId_tn_delete_metaLoc_batch.RLock()
   577  	defer tbl.getTxn().blockId_tn_delete_metaLoc_batch.RUnlock()
   578  
   579  	for blk, bats := range tbl.getTxn().blockId_tn_delete_metaLoc_batch.data {
   580  		//if blk is in partitionState.blks, it means that blk is persisted.
   581  		if state.BlockPersisted(blk) {
   582  			continue
   583  		}
   584  		for _, bat := range bats {
   585  			vs := vector.MustStrCol(bat.GetVector(0))
   586  			for _, metalLoc := range vs {
   587  				location, err := blockio.EncodeLocationFromString(metalLoc)
   588  				if err != nil {
   589  					return err
   590  				}
   591  				rowIdBat, release, err := blockio.LoadTombstoneColumns(
   592  					tbl.getTxn().proc.Ctx,
   593  					[]uint16{0},
   594  					nil,
   595  					tbl.getTxn().engine.fs,
   596  					location,
   597  					tbl.getTxn().proc.GetMPool())
   598  				if err != nil {
   599  					return err
   600  				}
   601  				defer release()
   602  				rowIds := vector.MustFixedCol[types.Rowid](rowIdBat.GetVector(0))
   603  				for _, rowId := range rowIds {
   604  					if deletesRowId != nil {
   605  						deletesRowId[rowId] = 0
   606  					}
   607  				}
   608  			}
   609  		}
   610  
   611  	}
   612  	return nil
   613  }
   614  
   615  func (tbl *txnTable) GetEngineType() engine.EngineType {
   616  	return engine.Disttae
   617  }
   618  
   619  func (tbl *txnTable) reset(newId uint64) {
   620  	//if the table is truncated first time, the table id is saved into the oldTableId
   621  	if tbl.oldTableId == 0 {
   622  		tbl.oldTableId = tbl.tableId
   623  	}
   624  	tbl.tableId = newId
   625  	tbl._partState.Store(nil)
   626  }
   627  
   628  func (tbl *txnTable) resetSnapshot() {
   629  	tbl._partState.Store(nil)
   630  }
   631  
   632  // return all unmodified blocks
   633  func (tbl *txnTable) Ranges(ctx context.Context, exprs []*plan.Expr) (ranges engine.Ranges, err error) {
   634  	start := time.Now()
   635  	seq := tbl.db.op.NextSequence()
   636  	trace.GetService().AddTxnDurationAction(
   637  		tbl.db.op,
   638  		client.RangesEvent,
   639  		seq,
   640  		tbl.tableId,
   641  		0,
   642  		nil)
   643  
   644  	defer func() {
   645  		cost := time.Since(start)
   646  
   647  		trace.GetService().AddTxnAction(
   648  			tbl.db.op,
   649  			client.RangesEvent,
   650  			seq,
   651  			tbl.tableId,
   652  			int64(ranges.Len()),
   653  			"blocks",
   654  			err)
   655  
   656  		trace.GetService().AddTxnDurationAction(
   657  			tbl.db.op,
   658  			client.RangesEvent,
   659  			seq,
   660  			tbl.tableId,
   661  			cost,
   662  			err)
   663  
   664  		v2.TxnTableRangeDurationHistogram.Observe(cost.Seconds())
   665  	}()
   666  
   667  	var blocks objectio.BlockInfoSlice
   668  	ranges = &blocks
   669  
   670  	// get the table's snapshot
   671  	var part *logtailreplay.PartitionState
   672  	if part, err = tbl.getPartitionState(ctx); err != nil {
   673  		return
   674  	}
   675  
   676  	blocks.AppendBlockInfo(objectio.EmptyBlockInfo)
   677  
   678  	if err = tbl.rangesOnePart(
   679  		ctx,
   680  		part,
   681  		tbl.GetTableDef(ctx),
   682  		exprs,
   683  		&blocks,
   684  		tbl.proc.Load(),
   685  	); err != nil {
   686  		return
   687  	}
   688  
   689  	return
   690  }
   691  
   692  // txn can read :
   693  //  1. snapshot data:
   694  //      1>. committed block data resides in S3.
   695  //      2>. partition state data resides in memory. read by partitionReader.
   696  
   697  //      deletes(rowids) for committed block exist in the following four places:
   698  //      1. in delta location formed by TN writing S3. read by blockReader.
   699  //      2. in CN's partition state, read by partitionReader.
   700  //  	3. in txn's workspace(txn.writes) being deleted by txn, read by partitionReader.
   701  //  	4. in delta location being deleted through CN writing S3, read by blockMergeReader.
   702  
   703  //  2. data in txn's workspace:
   704  //     1>.Raw batch data resides in txn.writes,read by partitionReader.
   705  //     2>.CN blocks resides in S3, read by blockReader.
   706  
   707  // rangesOnePart collect blocks which are visible to this txn,
   708  // include committed blocks and uncommitted blocks by CN writing S3.
   709  // notice that only clean blocks can be distributed into remote CNs.
   710  func (tbl *txnTable) rangesOnePart(
   711  	ctx context.Context,
   712  	state *logtailreplay.PartitionState, // snapshot state of this transaction
   713  	tableDef *plan.TableDef, // table definition (schema)
   714  	exprs []*plan.Expr, // filter expression
   715  	outBlocks *objectio.BlockInfoSlice, // output marshaled block list after filtering
   716  	proc *process.Process, // process of this transaction
   717  ) (err error) {
   718  
   719  	uncommittedObjects := tbl.collectUnCommittedObjects()
   720  	dirtyBlks := tbl.collectDirtyBlocks(state, uncommittedObjects)
   721  
   722  	done, err := tbl.tryFastFilterBlocks(
   723  		exprs, state, uncommittedObjects, dirtyBlks, outBlocks,
   724  		tbl.getTxn().engine.fs)
   725  	if err != nil {
   726  		return err
   727  	} else if done {
   728  		return nil
   729  	}
   730  
   731  	if done, err = tbl.tryFastRanges(
   732  		exprs, state, uncommittedObjects, dirtyBlks, outBlocks,
   733  		tbl.getTxn().engine.fs,
   734  	); err != nil {
   735  		return err
   736  	} else if done {
   737  		return nil
   738  	}
   739  
   740  	// for dynamic parameter, substitute param ref and const fold cast expression here to improve performance
   741  	newExprs, err := plan2.ConstandFoldList(exprs, tbl.proc.Load(), true)
   742  	if err == nil {
   743  		exprs = newExprs
   744  	}
   745  
   746  	var (
   747  		objMeta  objectio.ObjectMeta
   748  		zms      []objectio.ZoneMap
   749  		vecs     []*vector.Vector
   750  		skipObj  bool
   751  		auxIdCnt int32
   752  		s3BlkCnt uint32
   753  	)
   754  
   755  	defer func() {
   756  		for i := range vecs {
   757  			if vecs[i] != nil {
   758  				vecs[i].Free(proc.Mp())
   759  			}
   760  		}
   761  	}()
   762  
   763  	// check if expr is monotonic, if not, we can skip evaluating expr for each block
   764  	for _, expr := range exprs {
   765  		auxIdCnt += plan2.AssignAuxIdForExpr(expr, auxIdCnt)
   766  	}
   767  
   768  	columnMap := make(map[int]int)
   769  	if auxIdCnt > 0 {
   770  		zms = make([]objectio.ZoneMap, auxIdCnt)
   771  		vecs = make([]*vector.Vector, auxIdCnt)
   772  		plan2.GetColumnMapByExprs(exprs, tableDef, columnMap)
   773  	}
   774  
   775  	errCtx := errutil.ContextWithNoReport(ctx, true)
   776  
   777  	hasDeletes := len(dirtyBlks) > 0
   778  
   779  	if err = ForeachSnapshotObjects(
   780  		tbl.db.op.SnapshotTS(),
   781  		func(obj logtailreplay.ObjectInfo, isCommitted bool) (err2 error) {
   782  			var meta objectio.ObjectDataMeta
   783  			skipObj = false
   784  
   785  			s3BlkCnt += obj.BlkCnt()
   786  			if auxIdCnt > 0 {
   787  				v2.TxnRangesLoadedObjectMetaTotalCounter.Inc()
   788  				location := obj.ObjectLocation()
   789  				if objMeta, err2 = objectio.FastLoadObjectMeta(
   790  					errCtx, &location, false, tbl.getTxn().engine.fs,
   791  				); err2 != nil {
   792  					return
   793  				}
   794  
   795  				meta = objMeta.MustDataMeta()
   796  				// here we only eval expr on the object meta if it has more than one blocks
   797  				if meta.BlockCount() > 2 {
   798  					for _, expr := range exprs {
   799  						if !colexec.EvaluateFilterByZoneMap(
   800  							errCtx, proc, expr, meta, columnMap, zms, vecs,
   801  						) {
   802  							skipObj = true
   803  							break
   804  						}
   805  					}
   806  				}
   807  			}
   808  			if skipObj {
   809  				return
   810  			}
   811  
   812  			if obj.Rows() == 0 && meta.IsEmpty() {
   813  				location := obj.ObjectLocation()
   814  				if objMeta, err2 = objectio.FastLoadObjectMeta(
   815  					errCtx, &location, false, tbl.getTxn().engine.fs,
   816  				); err2 != nil {
   817  					return
   818  				}
   819  				meta = objMeta.MustDataMeta()
   820  			}
   821  
   822  			ForeachBlkInObjStatsList(true, meta, func(blk objectio.BlockInfo, blkMeta objectio.BlockObject) bool {
   823  				skipBlk := false
   824  
   825  				if auxIdCnt > 0 {
   826  					// eval filter expr on the block
   827  					for _, expr := range exprs {
   828  						if !colexec.EvaluateFilterByZoneMap(errCtx, proc, expr, blkMeta, columnMap, zms, vecs) {
   829  							skipBlk = true
   830  							break
   831  						}
   832  					}
   833  
   834  					// if the block is not needed, skip it
   835  					if skipBlk {
   836  						return true
   837  					}
   838  				}
   839  
   840  				blk.Sorted = obj.Sorted
   841  				blk.EntryState = obj.EntryState
   842  				blk.CommitTs = obj.CommitTS
   843  				if obj.HasDeltaLoc {
   844  					deltaLoc, commitTs, ok := state.GetBockDeltaLoc(blk.BlockID)
   845  					if ok {
   846  						blk.DeltaLoc = deltaLoc
   847  						blk.CommitTs = commitTs
   848  					}
   849  				}
   850  
   851  				if hasDeletes {
   852  					if _, ok := dirtyBlks[blk.BlockID]; !ok {
   853  						blk.CanRemote = true
   854  					}
   855  					blk.PartitionNum = -1
   856  					outBlocks.AppendBlockInfo(blk)
   857  					return true
   858  				}
   859  				// store the block in ranges
   860  				blk.CanRemote = true
   861  				blk.PartitionNum = -1
   862  				outBlocks.AppendBlockInfo(blk)
   863  
   864  				return true
   865  
   866  			},
   867  				obj.ObjectStats,
   868  			)
   869  			return
   870  		},
   871  		state,
   872  		uncommittedObjects...,
   873  	); err != nil {
   874  		return
   875  	}
   876  
   877  	bhit, btotal := outBlocks.Len()-1, int(s3BlkCnt)
   878  	v2.TaskSelBlockTotal.Add(float64(btotal))
   879  	v2.TaskSelBlockHit.Add(float64(btotal - bhit))
   880  	blockio.RecordBlockSelectivity(bhit, btotal)
   881  	if btotal > 0 {
   882  		v2.TxnRangeSizeHistogram.Observe(float64(bhit))
   883  		v2.TxnRangesBlockSelectivityHistogram.Observe(float64(bhit) / float64(btotal))
   884  	}
   885  	return
   886  }
   887  
   888  func (tbl *txnTable) collectUnCommittedObjects() []objectio.ObjectStats {
   889  	var unCommittedObjects []objectio.ObjectStats
   890  	tbl.getTxn().forEachTableWrites(
   891  		tbl.db.databaseId,
   892  		tbl.tableId,
   893  		tbl.getTxn().GetSnapshotWriteOffset(),
   894  		func(entry Entry) {
   895  			stats := objectio.ObjectStats{}
   896  			if entry.bat == nil || entry.bat.IsEmpty() {
   897  				return
   898  			}
   899  			if entry.typ == INSERT_TXN {
   900  				return
   901  			}
   902  			if entry.typ != INSERT ||
   903  				len(entry.bat.Attrs) < 2 ||
   904  				entry.bat.Attrs[1] != catalog.ObjectMeta_ObjectStats {
   905  				return
   906  			}
   907  			for i := 0; i < entry.bat.Vecs[1].Length(); i++ {
   908  				stats.UnMarshal(entry.bat.Vecs[1].GetBytesAt(i))
   909  				unCommittedObjects = append(unCommittedObjects, stats)
   910  			}
   911  		})
   912  
   913  	return unCommittedObjects
   914  }
   915  
   916  func (tbl *txnTable) collectDirtyBlocks(
   917  	state *logtailreplay.PartitionState,
   918  	uncommittedObjects []objectio.ObjectStats) map[types.Blockid]struct{} {
   919  	dirtyBlks := make(map[types.Blockid]struct{})
   920  	//collect partitionState.dirtyBlocks which may be invisible to this txn into dirtyBlks.
   921  	{
   922  		iter := state.NewDirtyBlocksIter()
   923  		for iter.Next() {
   924  			entry := iter.Entry()
   925  			//lazy load deletes for block.
   926  			dirtyBlks[entry] = struct{}{}
   927  		}
   928  		iter.Close()
   929  
   930  	}
   931  
   932  	//only collect dirty blocks in PartitionState.blocks into dirtyBlks.
   933  	for _, bid := range tbl.GetDirtyPersistedBlks(state) {
   934  		dirtyBlks[bid] = struct{}{}
   935  	}
   936  
   937  	if tbl.getTxn().hasDeletesOnUncommitedObject() {
   938  		ForeachBlkInObjStatsList(true, nil, func(blk objectio.BlockInfo, _ objectio.BlockObject) bool {
   939  			if tbl.getTxn().hasUncommittedDeletesOnBlock(&blk.BlockID) {
   940  				dirtyBlks[blk.BlockID] = struct{}{}
   941  			}
   942  			return true
   943  		}, uncommittedObjects...)
   944  	}
   945  
   946  	tbl.getTxn().forEachTableWrites(
   947  		tbl.db.databaseId,
   948  		tbl.tableId,
   949  		tbl.getTxn().GetSnapshotWriteOffset(),
   950  		func(entry Entry) {
   951  			// the CN workspace can only handle `INSERT` and `DELETE` operations. Other operations will be skipped,
   952  			// TODO Adjustments will be made here in the future
   953  			if entry.typ == DELETE || entry.typ == DELETE_TXN {
   954  				if entry.isGeneratedByTruncate() {
   955  					return
   956  				}
   957  				//deletes in tbl.writes maybe comes from PartitionState.rows or PartitionState.blocks.
   958  				if entry.fileName == "" &&
   959  					entry.tableId != catalog.MO_DATABASE_ID && entry.tableId != catalog.MO_TABLES_ID && entry.tableId != catalog.MO_COLUMNS_ID {
   960  					vs := vector.MustFixedCol[types.Rowid](entry.bat.GetVector(0))
   961  					for _, v := range vs {
   962  						id, _ := v.Decode()
   963  						dirtyBlks[id] = struct{}{}
   964  					}
   965  				}
   966  			}
   967  		})
   968  
   969  	return dirtyBlks
   970  }
   971  
   972  // tryFastFilterBlocks is going to replace the tryFastRanges completely soon, in progress now.
   973  func (tbl *txnTable) tryFastFilterBlocks(
   974  	exprs []*plan.Expr,
   975  	snapshot *logtailreplay.PartitionState,
   976  	uncommittedObjects []objectio.ObjectStats,
   977  	dirtyBlocks map[types.Blockid]struct{},
   978  	outBlocks *objectio.BlockInfoSlice,
   979  	fs fileservice.FileService) (done bool, err error) {
   980  	// TODO: refactor this code if composite key can be pushdown
   981  	if tbl.tableDef.Pkey.CompPkeyCol == nil {
   982  		return TryFastFilterBlocks(
   983  			tbl.db.op.SnapshotTS(),
   984  			tbl.tableDef,
   985  			exprs,
   986  			snapshot,
   987  			uncommittedObjects,
   988  			dirtyBlocks,
   989  			outBlocks,
   990  			fs,
   991  			tbl.proc.Load(),
   992  		)
   993  	}
   994  	return
   995  }
   996  
   997  // tryFastRanges only handle equal expression filter on zonemap and bloomfilter in tp scenario;
   998  // it filters out only a small number of blocks which should not be distributed to remote CNs.
   999  func (tbl *txnTable) tryFastRanges(
  1000  	exprs []*plan.Expr,
  1001  	snapshot *logtailreplay.PartitionState,
  1002  	uncommittedObjects []objectio.ObjectStats,
  1003  	dirtyBlocks map[types.Blockid]struct{},
  1004  	outBlocks *objectio.BlockInfoSlice,
  1005  	fs fileservice.FileService,
  1006  ) (done bool, err error) {
  1007  	if tbl.primaryIdx == -1 || len(exprs) == 0 {
  1008  		done = false
  1009  		return
  1010  	}
  1011  
  1012  	val, isVec := extractPKValueFromEqualExprs(
  1013  		tbl.tableDef,
  1014  		exprs,
  1015  		tbl.primaryIdx,
  1016  		tbl.proc.Load(),
  1017  		tbl.getTxn().engine.packerPool,
  1018  	)
  1019  	if len(val) == 0 {
  1020  		// TODO: refactor this code if composite key can be pushdown
  1021  		return TryFastFilterBlocks(
  1022  			tbl.db.op.SnapshotTS(),
  1023  			tbl.tableDef,
  1024  			exprs,
  1025  			snapshot,
  1026  			uncommittedObjects,
  1027  			dirtyBlocks,
  1028  			outBlocks,
  1029  			fs,
  1030  			tbl.proc.Load(),
  1031  		)
  1032  	}
  1033  
  1034  	var (
  1035  		meta     objectio.ObjectDataMeta
  1036  		bf       objectio.BloomFilter
  1037  		blockCnt uint32
  1038  		zmTotal  float64
  1039  		zmHit    float64
  1040  	)
  1041  
  1042  	defer func() {
  1043  		if zmTotal > 0 {
  1044  			v2.TxnFastRangesZMapSelectivityHistogram.Observe(zmHit / zmTotal)
  1045  		}
  1046  	}()
  1047  
  1048  	var vec *vector.Vector
  1049  	if isVec {
  1050  		vec = vector.NewVec(types.T_any.ToType())
  1051  		_ = vec.UnmarshalBinary(val)
  1052  	}
  1053  
  1054  	hasDeletes := len(dirtyBlocks) > 0
  1055  
  1056  	if err = ForeachSnapshotObjects(
  1057  		tbl.db.op.SnapshotTS(),
  1058  		func(obj logtailreplay.ObjectInfo, isCommitted bool) (err2 error) {
  1059  			zmTotal++
  1060  			blockCnt += obj.BlkCnt()
  1061  			var zmCkecked bool
  1062  			// if the object info contains a pk zonemap, fast-check with the zonemap
  1063  			if !obj.ZMIsEmpty() {
  1064  				if isVec {
  1065  					if !obj.SortKeyZoneMap().AnyIn(vec) {
  1066  						zmHit++
  1067  						return
  1068  					}
  1069  				} else {
  1070  					if !obj.SortKeyZoneMap().ContainsKey(val) {
  1071  						zmHit++
  1072  						return
  1073  					}
  1074  				}
  1075  				zmCkecked = true
  1076  			}
  1077  
  1078  			var objMeta objectio.ObjectMeta
  1079  			location := obj.Location()
  1080  
  1081  			// load object metadata
  1082  			v2.TxnRangesLoadedObjectMetaTotalCounter.Inc()
  1083  			if objMeta, err2 = objectio.FastLoadObjectMeta(
  1084  				tbl.proc.Load().Ctx, &location, false, fs,
  1085  			); err2 != nil {
  1086  				return
  1087  			}
  1088  
  1089  			// reset bloom filter to nil for each object
  1090  			meta = objMeta.MustDataMeta()
  1091  
  1092  			// check whether the object is skipped by zone map
  1093  			// If object zone map doesn't contains the pk value, we need to check bloom filter
  1094  			if !zmCkecked {
  1095  				if isVec {
  1096  					if !meta.MustGetColumn(uint16(tbl.primaryIdx)).ZoneMap().AnyIn(vec) {
  1097  						return
  1098  					}
  1099  				} else {
  1100  					if !meta.MustGetColumn(uint16(tbl.primaryIdx)).ZoneMap().ContainsKey(val) {
  1101  						return
  1102  					}
  1103  				}
  1104  			}
  1105  
  1106  			bf = nil
  1107  			if bf, err2 = objectio.LoadBFWithMeta(
  1108  				tbl.proc.Load().Ctx, meta, location, fs,
  1109  			); err2 != nil {
  1110  				return
  1111  			}
  1112  
  1113  			var blkIdx int
  1114  			blockCnt := int(meta.BlockCount())
  1115  			if !isVec {
  1116  				blkIdx = sort.Search(blockCnt, func(j int) bool {
  1117  					return meta.GetBlockMeta(uint32(j)).MustGetColumn(uint16(tbl.primaryIdx)).ZoneMap().AnyGEByValue(val)
  1118  				})
  1119  			}
  1120  			if blkIdx >= blockCnt {
  1121  				return
  1122  			}
  1123  			for ; blkIdx < blockCnt; blkIdx++ {
  1124  				blkMeta := meta.GetBlockMeta(uint32(blkIdx))
  1125  				zm := blkMeta.MustGetColumn(uint16(tbl.primaryIdx)).ZoneMap()
  1126  				if !isVec && !zm.AnyLEByValue(val) {
  1127  					break
  1128  				}
  1129  				if isVec {
  1130  					if !zm.AnyIn(vec) {
  1131  						continue
  1132  					}
  1133  				} else {
  1134  					if !zm.ContainsKey(val) {
  1135  						continue
  1136  					}
  1137  				}
  1138  
  1139  				blkBf := bf.GetBloomFilter(uint32(blkIdx))
  1140  				blkBfIdx := index.NewEmptyBinaryFuseFilter()
  1141  				if err2 = index.DecodeBloomFilter(blkBfIdx, blkBf); err2 != nil {
  1142  					return
  1143  				}
  1144  				var exist bool
  1145  				if isVec {
  1146  					lowerBound, upperBound := zm.SubVecIn(vec)
  1147  					if exist = blkBfIdx.MayContainsAny(vec, lowerBound, upperBound); !exist {
  1148  						continue
  1149  					}
  1150  				} else {
  1151  					if exist, err2 = blkBfIdx.MayContainsKey(val); err2 != nil {
  1152  						return
  1153  					} else if !exist {
  1154  						continue
  1155  					}
  1156  				}
  1157  
  1158  				name := obj.ObjectName()
  1159  				loc := objectio.BuildLocation(name, obj.Extent(), blkMeta.GetRows(), uint16(blkIdx))
  1160  				blk := objectio.BlockInfo{
  1161  					BlockID:   *objectio.BuildObjectBlockid(name, uint16(blkIdx)),
  1162  					SegmentID: name.SegmentId(),
  1163  					MetaLoc:   objectio.ObjectLocation(loc),
  1164  				}
  1165  
  1166  				blk.Sorted = obj.Sorted
  1167  				blk.EntryState = obj.EntryState
  1168  				blk.CommitTs = obj.CommitTS
  1169  				if obj.HasDeltaLoc {
  1170  					deltaLoc, commitTs, ok := snapshot.GetBockDeltaLoc(blk.BlockID)
  1171  					if ok {
  1172  						blk.DeltaLoc = deltaLoc
  1173  						blk.CommitTs = commitTs
  1174  					}
  1175  				}
  1176  
  1177  				if hasDeletes {
  1178  					if _, ok := dirtyBlocks[blk.BlockID]; !ok {
  1179  						blk.CanRemote = true
  1180  					}
  1181  					blk.PartitionNum = -1
  1182  					outBlocks.AppendBlockInfo(blk)
  1183  					return
  1184  				}
  1185  				// store the block in ranges
  1186  				blk.CanRemote = true
  1187  				blk.PartitionNum = -1
  1188  				outBlocks.AppendBlockInfo(blk)
  1189  			}
  1190  
  1191  			return
  1192  		},
  1193  		snapshot,
  1194  		uncommittedObjects...,
  1195  	); err != nil {
  1196  		return
  1197  	}
  1198  
  1199  	done = true
  1200  	bhit, btotal := outBlocks.Len()-1, int(blockCnt)
  1201  	v2.TaskSelBlockTotal.Add(float64(btotal))
  1202  	v2.TaskSelBlockHit.Add(float64(btotal - bhit))
  1203  	blockio.RecordBlockSelectivity(bhit, btotal)
  1204  	if btotal > 0 {
  1205  		v2.TxnFastRangeSizeHistogram.Observe(float64(bhit))
  1206  		v2.TxnFastRangesBlockSelectivityHistogram.Observe(float64(bhit) / float64(btotal))
  1207  	}
  1208  
  1209  	return
  1210  }
  1211  
  1212  // the return defs has no rowid column
  1213  func (tbl *txnTable) TableDefs(ctx context.Context) ([]engine.TableDef, error) {
  1214  	//return tbl.defs, nil
  1215  	// I don't understand why the logic now is not to get all the tableDef. Don't understand.
  1216  	// copy from tae's logic
  1217  	defs := make([]engine.TableDef, 0, len(tbl.defs))
  1218  	defs = append(defs, &engine.VersionDef{Version: tbl.version})
  1219  	if tbl.comment != "" {
  1220  		commentDef := new(engine.CommentDef)
  1221  		commentDef.Comment = tbl.comment
  1222  		defs = append(defs, commentDef)
  1223  	}
  1224  	if tbl.partitioned > 0 || tbl.partition != "" {
  1225  		partitionDef := new(engine.PartitionDef)
  1226  		partitionDef.Partitioned = tbl.partitioned
  1227  		partitionDef.Partition = tbl.partition
  1228  		defs = append(defs, partitionDef)
  1229  	}
  1230  
  1231  	if tbl.viewdef != "" {
  1232  		viewDef := new(engine.ViewDef)
  1233  		viewDef.View = tbl.viewdef
  1234  		defs = append(defs, viewDef)
  1235  	}
  1236  	if len(tbl.constraint) > 0 {
  1237  		c := &engine.ConstraintDef{}
  1238  		err := c.UnmarshalBinary(tbl.constraint)
  1239  		if err != nil {
  1240  			return nil, err
  1241  		}
  1242  		defs = append(defs, c)
  1243  	}
  1244  	for i, def := range tbl.defs {
  1245  		if attr, ok := def.(*engine.AttributeDef); ok {
  1246  			if attr.Attr.Name != catalog.Row_ID {
  1247  				defs = append(defs, tbl.defs[i])
  1248  			}
  1249  		}
  1250  	}
  1251  	pro := new(engine.PropertiesDef)
  1252  	pro.Properties = append(pro.Properties, engine.Property{
  1253  		Key:   catalog.SystemRelAttr_Kind,
  1254  		Value: string(tbl.relKind),
  1255  	})
  1256  	if tbl.createSql != "" {
  1257  		pro.Properties = append(pro.Properties, engine.Property{
  1258  			Key:   catalog.SystemRelAttr_CreateSQL,
  1259  			Value: tbl.createSql,
  1260  		})
  1261  	}
  1262  	defs = append(defs, pro)
  1263  	return defs, nil
  1264  }
  1265  
  1266  func (tbl *txnTable) GetTableDef(ctx context.Context) *plan.TableDef {
  1267  	if tbl.tableDef == nil {
  1268  		var clusterByDef *plan.ClusterByDef
  1269  		var cols []*plan.ColDef
  1270  		var defs []*plan.TableDef_DefType
  1271  		var properties []*plan.Property
  1272  		var TableType string
  1273  		var Createsql string
  1274  		var partitionInfo *plan.PartitionByDef
  1275  		var viewSql *plan.ViewDef
  1276  		var foreignKeys []*plan.ForeignKeyDef
  1277  		var primarykey *plan.PrimaryKeyDef
  1278  		var indexes []*plan.IndexDef
  1279  		var refChildTbls []uint64
  1280  		var hasRowId bool
  1281  
  1282  		i := int32(0)
  1283  		name2index := make(map[string]int32)
  1284  		for _, def := range tbl.defs {
  1285  			if attr, ok := def.(*engine.AttributeDef); ok {
  1286  				name2index[attr.Attr.Name] = i
  1287  				cols = append(cols, &plan.ColDef{
  1288  					ColId: attr.Attr.ID,
  1289  					Name:  attr.Attr.Name,
  1290  					Typ: plan.Type{
  1291  						Id:          int32(attr.Attr.Type.Oid),
  1292  						Width:       attr.Attr.Type.Width,
  1293  						Scale:       attr.Attr.Type.Scale,
  1294  						AutoIncr:    attr.Attr.AutoIncrement,
  1295  						Table:       tbl.tableName,
  1296  						NotNullable: attr.Attr.Default != nil && !attr.Attr.Default.NullAbility,
  1297  						Enumvalues:  attr.Attr.EnumVlaues,
  1298  					},
  1299  					Primary:   attr.Attr.Primary,
  1300  					Default:   attr.Attr.Default,
  1301  					OnUpdate:  attr.Attr.OnUpdate,
  1302  					Comment:   attr.Attr.Comment,
  1303  					ClusterBy: attr.Attr.ClusterBy,
  1304  					Hidden:    attr.Attr.IsHidden,
  1305  					Seqnum:    uint32(attr.Attr.Seqnum),
  1306  				})
  1307  				if attr.Attr.ClusterBy {
  1308  					clusterByDef = &plan.ClusterByDef{
  1309  						Name: attr.Attr.Name,
  1310  					}
  1311  				}
  1312  				if attr.Attr.Name == catalog.Row_ID {
  1313  					hasRowId = true
  1314  				}
  1315  				i++
  1316  			}
  1317  		}
  1318  
  1319  		if tbl.comment != "" {
  1320  			properties = append(properties, &plan.Property{
  1321  				Key:   catalog.SystemRelAttr_Comment,
  1322  				Value: tbl.comment,
  1323  			})
  1324  		}
  1325  
  1326  		if tbl.partitioned > 0 {
  1327  			p := &plan.PartitionByDef{}
  1328  			err := p.UnMarshalPartitionInfo(([]byte)(tbl.partition))
  1329  			if err != nil {
  1330  				//panic(fmt.Sprintf("cannot unmarshal partition metadata information: %s", err))
  1331  				return nil
  1332  			}
  1333  			partitionInfo = p
  1334  		}
  1335  
  1336  		if tbl.viewdef != "" {
  1337  			viewSql = &plan.ViewDef{
  1338  				View: tbl.viewdef,
  1339  			}
  1340  		}
  1341  
  1342  		if len(tbl.constraint) > 0 {
  1343  			c := &engine.ConstraintDef{}
  1344  			err := c.UnmarshalBinary(tbl.constraint)
  1345  			if err != nil {
  1346  				//panic(fmt.Sprintf("cannot unmarshal table constraint information: %s", err))
  1347  				return nil
  1348  			}
  1349  			for _, ct := range c.Cts {
  1350  				switch k := ct.(type) {
  1351  				case *engine.IndexDef:
  1352  					indexes = k.Indexes
  1353  				case *engine.ForeignKeyDef:
  1354  					foreignKeys = k.Fkeys
  1355  				case *engine.RefChildTableDef:
  1356  					refChildTbls = k.Tables
  1357  				case *engine.PrimaryKeyDef:
  1358  					primarykey = k.Pkey
  1359  				case *engine.StreamConfigsDef:
  1360  					properties = append(properties, k.Configs...)
  1361  				}
  1362  			}
  1363  		}
  1364  
  1365  		properties = append(properties, &plan.Property{
  1366  			Key:   catalog.SystemRelAttr_Kind,
  1367  			Value: tbl.relKind,
  1368  		})
  1369  		TableType = tbl.relKind
  1370  
  1371  		if tbl.createSql != "" {
  1372  			properties = append(properties, &plan.Property{
  1373  				Key:   catalog.SystemRelAttr_CreateSQL,
  1374  				Value: tbl.createSql,
  1375  			})
  1376  			Createsql = tbl.createSql
  1377  		}
  1378  
  1379  		if len(properties) > 0 {
  1380  			defs = append(defs, &plan.TableDef_DefType{
  1381  				Def: &plan.TableDef_DefType_Properties{
  1382  					Properties: &plan.PropertiesDef{
  1383  						Properties: properties,
  1384  					},
  1385  				},
  1386  			})
  1387  		}
  1388  
  1389  		if primarykey != nil && primarykey.PkeyColName == catalog.CPrimaryKeyColName {
  1390  			primarykey.CompPkeyCol = plan2.GetColDefFromTable(cols, catalog.CPrimaryKeyColName)
  1391  		}
  1392  		if clusterByDef != nil && util.JudgeIsCompositeClusterByColumn(clusterByDef.Name) {
  1393  			clusterByDef.CompCbkeyCol = plan2.GetColDefFromTable(cols, clusterByDef.Name)
  1394  		}
  1395  		if !hasRowId {
  1396  			rowIdCol := plan2.MakeRowIdColDef()
  1397  			cols = append(cols, rowIdCol)
  1398  		}
  1399  
  1400  		tbl.tableDef = &plan.TableDef{
  1401  			TblId:         tbl.tableId,
  1402  			Name:          tbl.tableName,
  1403  			Cols:          cols,
  1404  			Name2ColIndex: name2index,
  1405  			Defs:          defs,
  1406  			TableType:     TableType,
  1407  			Createsql:     Createsql,
  1408  			Pkey:          primarykey,
  1409  			ViewSql:       viewSql,
  1410  			Partition:     partitionInfo,
  1411  			Fkeys:         foreignKeys,
  1412  			RefChildTbls:  refChildTbls,
  1413  			ClusterBy:     clusterByDef,
  1414  			Indexes:       indexes,
  1415  			Version:       tbl.version,
  1416  		}
  1417  	}
  1418  	return tbl.tableDef
  1419  }
  1420  
  1421  func (tbl *txnTable) CopyTableDef(ctx context.Context) *plan.TableDef {
  1422  	tbl.GetTableDef(ctx)
  1423  	return plan2.DeepCopyTableDef(tbl.tableDef, true)
  1424  }
  1425  
  1426  func (tbl *txnTable) UpdateConstraint(ctx context.Context, c *engine.ConstraintDef) error {
  1427  	if tbl.db.op.IsSnapOp() {
  1428  		return moerr.NewInternalErrorNoCtx("cannot update table constraint in snapshot operation")
  1429  	}
  1430  	ct, err := c.MarshalBinary()
  1431  	if err != nil {
  1432  		return err
  1433  	}
  1434  	bat, err := genTableConstraintTuple(tbl.tableId, tbl.db.databaseId, tbl.tableName, tbl.db.databaseName,
  1435  		ct, tbl.getTxn().proc.Mp())
  1436  	if err != nil {
  1437  		return err
  1438  	}
  1439  	if err = tbl.getTxn().WriteBatch(UPDATE, 0, catalog.MO_CATALOG_ID, catalog.MO_TABLES_ID,
  1440  		catalog.MO_CATALOG, catalog.MO_TABLES, bat, tbl.getTxn().tnStores[0], -1, false, false); err != nil {
  1441  		bat.Clean(tbl.getTxn().proc.Mp())
  1442  		return err
  1443  	}
  1444  	tbl.constraint = ct
  1445  	tbl.tableDef = nil
  1446  	tbl.GetTableDef(ctx)
  1447  	return nil
  1448  }
  1449  
  1450  func (tbl *txnTable) AlterTable(ctx context.Context, c *engine.ConstraintDef, constraint [][]byte) error {
  1451  	if tbl.db.op.IsSnapOp() {
  1452  		return moerr.NewInternalErrorNoCtx("cannot alter table in snapshot operation")
  1453  	}
  1454  	ct, err := c.MarshalBinary()
  1455  	if err != nil {
  1456  		return err
  1457  	}
  1458  	bat, err := genTableAlterTuple(constraint, tbl.getTxn().proc.Mp())
  1459  	if err != nil {
  1460  		return err
  1461  	}
  1462  	if err = tbl.getTxn().WriteBatch(ALTER, 0, catalog.MO_CATALOG_ID, catalog.MO_TABLES_ID,
  1463  		catalog.MO_CATALOG, catalog.MO_TABLES, bat, tbl.getTxn().tnStores[0], -1, false, false); err != nil {
  1464  		bat.Clean(tbl.getTxn().proc.Mp())
  1465  		return err
  1466  	}
  1467  	tbl.constraint = ct
  1468  	// add tbl.partition = partition
  1469  
  1470  	tbl.tableDef = nil
  1471  
  1472  	// update TableDef
  1473  	tbl.GetTableDef(ctx)
  1474  	return nil
  1475  }
  1476  
  1477  func (tbl *txnTable) TableRenameInTxn(ctx context.Context, constraint [][]byte) error {
  1478  	if tbl.db.op.IsSnapOp() {
  1479  		return moerr.NewInternalErrorNoCtx("cannot rename table in snapshot operation")
  1480  	}
  1481  	// 1. delete cn metadata of table
  1482  	accountId, userId, roleId, err := getAccessInfo(ctx)
  1483  	if err != nil {
  1484  		return err
  1485  	}
  1486  	databaseId := tbl.GetDBID(ctx)
  1487  	db := tbl.db
  1488  	oldTableName := tbl.tableName
  1489  
  1490  	var id uint64
  1491  	var rowid types.Rowid
  1492  	var rowids []types.Rowid
  1493  	key := genTableKey(accountId, tbl.tableName, databaseId)
  1494  	if value, ok := tbl.db.getTxn().createMap.Load(key); ok {
  1495  		tbl.db.getTxn().createMap.Delete(key)
  1496  		table := value.(*txnTable)
  1497  		id = table.tableId
  1498  		rowid = table.rowid
  1499  		rowids = table.rowids
  1500  		if tbl != table {
  1501  			panic("The table object in createMap should be the current table object")
  1502  		}
  1503  	} else if value, ok := tbl.db.getTxn().tableCache.tableMap.Load(key); ok {
  1504  		table := value.(*txnTable)
  1505  		id = table.tableId
  1506  		rowid = table.rowid
  1507  		rowids = table.rowids
  1508  		if tbl != table {
  1509  			panic("The table object in tableCache should be the current table object")
  1510  		}
  1511  		tbl.db.getTxn().tableCache.tableMap.Delete(key)
  1512  	} else {
  1513  		// I think it is unnecessary to make a judgment on this branch because the table is already in use, so it must be in the cache
  1514  		item := &cache.TableItem{
  1515  			Name:       tbl.tableName,
  1516  			DatabaseId: databaseId,
  1517  			AccountId:  accountId,
  1518  			Ts:         db.op.SnapshotTS(),
  1519  		}
  1520  		if ok := tbl.db.getTxn().engine.getLatestCatalogCache().GetTable(item); !ok {
  1521  			return moerr.GetOkExpectedEOB()
  1522  		}
  1523  		id = item.Id
  1524  		rowid = item.Rowid
  1525  		rowids = item.Rowids
  1526  	}
  1527  
  1528  	bat, err := genDropTableTuple(rowid, id, db.databaseId, tbl.tableName,
  1529  		db.databaseName, tbl.db.getTxn().proc.Mp())
  1530  	if err != nil {
  1531  		return err
  1532  	}
  1533  	for _, store := range tbl.db.getTxn().tnStores {
  1534  		if err := tbl.db.getTxn().WriteBatch(DELETE_TXN, 0, catalog.MO_CATALOG_ID, catalog.MO_TABLES_ID,
  1535  			catalog.MO_CATALOG, catalog.MO_TABLES, bat, store, -1, false, false); err != nil {
  1536  			bat.Clean(tbl.db.getTxn().proc.Mp())
  1537  			return err
  1538  		}
  1539  	}
  1540  
  1541  	// Add writeBatch(delete,mo_columns) to filter table in mo_columns.
  1542  	// Every row in writeBatch(delete,mo_columns) needs rowid
  1543  	for _, rid := range rowids {
  1544  		bat, err = genDropColumnTuple(rid, tbl.db.getTxn().proc.Mp())
  1545  		if err != nil {
  1546  			return err
  1547  		}
  1548  		for _, store := range tbl.db.getTxn().tnStores {
  1549  			if err = tbl.db.getTxn().WriteBatch(DELETE_TXN, 0, catalog.MO_CATALOG_ID, catalog.MO_COLUMNS_ID,
  1550  				catalog.MO_CATALOG, catalog.MO_COLUMNS, bat, store, -1, false, false); err != nil {
  1551  				bat.Clean(tbl.db.getTxn().proc.Mp())
  1552  				return err
  1553  			}
  1554  		}
  1555  	}
  1556  	tbl.db.getTxn().deletedTableMap.Store(key, id)
  1557  
  1558  	//------------------------------------------------------------------------------------------------------------------
  1559  	// 2. send alter message to DN
  1560  	bat, err = genTableAlterTuple(constraint, tbl.db.getTxn().proc.Mp())
  1561  	if err != nil {
  1562  		return err
  1563  	}
  1564  	if err = tbl.db.getTxn().WriteBatch(ALTER, 0, catalog.MO_CATALOG_ID, catalog.MO_TABLES_ID,
  1565  		catalog.MO_CATALOG, catalog.MO_TABLES, bat, tbl.db.getTxn().tnStores[0], -1, false, false); err != nil {
  1566  		bat.Clean(tbl.db.getTxn().proc.Mp())
  1567  		return err
  1568  	}
  1569  
  1570  	req := &api.AlterTableReq{}
  1571  	if err = req.Unmarshal(constraint[0]); err != nil {
  1572  		return err
  1573  	} else {
  1574  		rename_table := req.Operation.(*api.AlterTableReq_RenameTable)
  1575  		newTblName := rename_table.RenameTable.NewName
  1576  		tbl.tableName = newTblName
  1577  	}
  1578  	tbl.tableDef = nil
  1579  	tbl.GetTableDef(ctx)
  1580  
  1581  	//------------------------------------------------------------------------------------------------------------------
  1582  	// 3. insert new table metadata
  1583  	newtbl := new(txnTable)
  1584  	newtbl.accountId = accountId
  1585  
  1586  	newRowId, err := tbl.db.getTxn().allocateID(ctx)
  1587  	if err != nil {
  1588  		return err
  1589  	}
  1590  	newtbl.rowid = types.DecodeFixed[types.Rowid](types.EncodeSlice([]uint64{newRowId}))
  1591  	newtbl.comment = tbl.comment
  1592  	newtbl.relKind = tbl.relKind
  1593  	newtbl.createSql = tbl.createSql
  1594  	newtbl.viewdef = tbl.viewdef
  1595  	newtbl.partitioned = tbl.partitioned
  1596  	newtbl.partition = tbl.partition
  1597  	newtbl.constraint = tbl.constraint
  1598  	newtbl.primaryIdx = tbl.primaryIdx
  1599  	newtbl.primarySeqnum = tbl.primarySeqnum
  1600  	newtbl.clusterByIdx = tbl.clusterByIdx
  1601  	newtbl.db = db
  1602  	newtbl.defs = tbl.defs
  1603  	newtbl.tableName = tbl.tableName
  1604  	newtbl.tableId = tbl.tableId
  1605  	newtbl.CopyTableDef(ctx)
  1606  
  1607  	{
  1608  		sql := getSql(ctx)
  1609  		bat, err := genCreateTableTuple(newtbl, sql, accountId, userId, roleId, newtbl.tableName,
  1610  			newtbl.tableId, db.databaseId, db.databaseName, newtbl.rowid, true, tbl.db.getTxn().proc.Mp())
  1611  		if err != nil {
  1612  			return err
  1613  		}
  1614  		for _, store := range tbl.db.getTxn().tnStores {
  1615  			if err := tbl.db.getTxn().WriteBatch(INSERT_TXN, 0, catalog.MO_CATALOG_ID, catalog.MO_TABLES_ID,
  1616  				catalog.MO_CATALOG, catalog.MO_TABLES, bat, store, -1, true, false); err != nil {
  1617  				bat.Clean(tbl.db.getTxn().proc.Mp())
  1618  				return err
  1619  			}
  1620  		}
  1621  	}
  1622  
  1623  	cols, err := genColumns(accountId, newtbl.tableName, db.databaseName, newtbl.tableId, db.databaseId, newtbl.defs)
  1624  	if err != nil {
  1625  		return err
  1626  	}
  1627  
  1628  	newtbl.rowids = make([]types.Rowid, len(cols))
  1629  	for i, col := range cols {
  1630  		newtbl.rowids[i] = tbl.db.getTxn().genRowId()
  1631  		bat, err := genCreateColumnTuple(col, newtbl.rowids[i], true, tbl.db.getTxn().proc.Mp())
  1632  		if err != nil {
  1633  			return err
  1634  		}
  1635  		for _, store := range tbl.db.getTxn().tnStores {
  1636  			if err := tbl.db.getTxn().WriteBatch(INSERT_TXN, 0, catalog.MO_CATALOG_ID, catalog.MO_COLUMNS_ID,
  1637  				catalog.MO_CATALOG, catalog.MO_COLUMNS, bat, store, -1, true, false); err != nil {
  1638  				bat.Clean(tbl.db.getTxn().proc.Mp())
  1639  				return err
  1640  			}
  1641  		}
  1642  		if col.constraintType == catalog.SystemColPKConstraint {
  1643  			newtbl.primaryIdx = i
  1644  			newtbl.primarySeqnum = i
  1645  		}
  1646  		if col.isClusterBy == 1 {
  1647  			newtbl.clusterByIdx = i
  1648  		}
  1649  	}
  1650  
  1651  	newkey := genTableKey(accountId, newtbl.tableName, databaseId)
  1652  	newtbl.getTxn().addCreateTable(newkey, newtbl)
  1653  	newtbl.getTxn().deletedTableMap.Delete(newkey)
  1654  	//---------------------------------------------------------------------------------
  1655  	for i := 0; i < len(newtbl.getTxn().writes); i++ {
  1656  		if newtbl.getTxn().writes[i].tableId == catalog.MO_DATABASE_ID ||
  1657  			newtbl.getTxn().writes[i].tableId == catalog.MO_TABLES_ID ||
  1658  			newtbl.getTxn().writes[i].tableId == catalog.MO_COLUMNS_ID {
  1659  			continue
  1660  		}
  1661  
  1662  		if newtbl.getTxn().writes[i].tableName == oldTableName {
  1663  			newtbl.getTxn().writes[i].tableName = tbl.tableName
  1664  			logutil.Infof("copy table '%s' has been rename to '%s' in txn", oldTableName, tbl.tableName)
  1665  		}
  1666  	}
  1667  	//---------------------------------------------------------------------------------
  1668  	return nil
  1669  }
  1670  
  1671  func (tbl *txnTable) TableColumns(ctx context.Context) ([]*engine.Attribute, error) {
  1672  	var attrs []*engine.Attribute
  1673  	for _, def := range tbl.defs {
  1674  		if attr, ok := def.(*engine.AttributeDef); ok {
  1675  			attrs = append(attrs, &attr.Attr)
  1676  		}
  1677  	}
  1678  	return attrs, nil
  1679  }
  1680  
  1681  func (tbl *txnTable) GetPrimaryKeys(ctx context.Context) ([]*engine.Attribute, error) {
  1682  	attrs := make([]*engine.Attribute, 0, 1)
  1683  	for _, def := range tbl.defs {
  1684  		if attr, ok := def.(*engine.AttributeDef); ok {
  1685  			if attr.Attr.Primary {
  1686  				attrs = append(attrs, &attr.Attr)
  1687  			}
  1688  		}
  1689  	}
  1690  	return attrs, nil
  1691  }
  1692  
  1693  func (tbl *txnTable) GetHideKeys(ctx context.Context) ([]*engine.Attribute, error) {
  1694  	attrs := make([]*engine.Attribute, 0, 1)
  1695  	attrs = append(attrs, &engine.Attribute{
  1696  		IsHidden: true,
  1697  		IsRowId:  true,
  1698  		Name:     catalog.Row_ID,
  1699  		Type:     types.New(types.T_Rowid, 0, 0),
  1700  		Primary:  true,
  1701  	})
  1702  	return attrs, nil
  1703  }
  1704  
  1705  func (tbl *txnTable) Write(ctx context.Context, bat *batch.Batch) error {
  1706  	if tbl.db.op.IsSnapOp() {
  1707  		return moerr.NewInternalErrorNoCtx("write operation is not allowed in snapshot transaction")
  1708  	}
  1709  	if bat == nil || bat.RowCount() == 0 {
  1710  		return nil
  1711  	}
  1712  	// for writing S3 Block
  1713  	if bat.Attrs[0] == catalog.BlockMeta_BlockInfo {
  1714  		tbl.getTxn().hasS3Op.Store(true)
  1715  		//bocks maybe come from different S3 object, here we just need to make sure fileName is not Nil.
  1716  		fileName := objectio.DecodeBlockInfo(bat.Vecs[0].GetBytesAt(0)).MetaLocation().Name().String()
  1717  		return tbl.getTxn().WriteFile(
  1718  			INSERT,
  1719  			tbl.accountId,
  1720  			tbl.db.databaseId,
  1721  			tbl.tableId,
  1722  			tbl.db.databaseName,
  1723  			tbl.tableName,
  1724  			fileName,
  1725  			bat,
  1726  			tbl.getTxn().tnStores[0])
  1727  	}
  1728  	ibat, err := util.CopyBatch(bat, tbl.getTxn().proc)
  1729  	if err != nil {
  1730  		return err
  1731  	}
  1732  	if err := tbl.getTxn().WriteBatch(
  1733  		INSERT,
  1734  		tbl.accountId,
  1735  		tbl.db.databaseId,
  1736  		tbl.tableId,
  1737  		tbl.db.databaseName,
  1738  		tbl.tableName,
  1739  		ibat,
  1740  		tbl.getTxn().tnStores[0],
  1741  		tbl.primaryIdx,
  1742  		false,
  1743  		false); err != nil {
  1744  		ibat.Clean(tbl.getTxn().proc.Mp())
  1745  		return err
  1746  	}
  1747  	return tbl.getTxn().dumpBatch(tbl.getTxn().GetSnapshotWriteOffset())
  1748  }
  1749  
  1750  func (tbl *txnTable) Update(ctx context.Context, bat *batch.Batch) error {
  1751  	if tbl.db.op.IsSnapOp() {
  1752  		return moerr.NewInternalErrorNoCtx("update operation is not allowed in snapshot transaction")
  1753  	}
  1754  	return nil
  1755  }
  1756  
  1757  //	blkId(string)     deltaLoc(string)                   type(int)
  1758  //
  1759  // |-----------|-----------------------------------|----------------|
  1760  // |  blk_id   |   batch.Marshal(deltaLoc)         |  FlushDeltaLoc | TN Block
  1761  // |  blk_id   |   batch.Marshal(uint32 offset)    |  CNBlockOffset | CN Block
  1762  // |  blk_id   |   batch.Marshal(rowId)            |  RawRowIdBatch | TN Blcok
  1763  // |  blk_id   |   batch.Marshal(uint32 offset)    | RawBatchOffset | RawBatch (in txn workspace)
  1764  func (tbl *txnTable) EnhanceDelete(bat *batch.Batch, name string) error {
  1765  	blkId, typ_str := objectio.Str2Blockid(name[:len(name)-2]), string(name[len(name)-1])
  1766  	typ, err := strconv.ParseInt(typ_str, 10, 64)
  1767  	if err != nil {
  1768  		return err
  1769  	}
  1770  	switch typ {
  1771  	case deletion.FlushDeltaLoc:
  1772  		tbl.getTxn().hasS3Op.Store(true)
  1773  		location, err := blockio.EncodeLocationFromString(bat.Vecs[0].GetStringAt(0))
  1774  		if err != nil {
  1775  			return err
  1776  		}
  1777  		fileName := location.Name().String()
  1778  		copBat, err := util.CopyBatch(bat, tbl.getTxn().proc)
  1779  		if err != nil {
  1780  			return err
  1781  		}
  1782  		if err := tbl.getTxn().WriteFile(DELETE, tbl.accountId, tbl.db.databaseId, tbl.tableId,
  1783  			tbl.db.databaseName, tbl.tableName, fileName, copBat, tbl.getTxn().tnStores[0]); err != nil {
  1784  			return err
  1785  		}
  1786  
  1787  		tbl.getTxn().blockId_tn_delete_metaLoc_batch.RWMutex.Lock()
  1788  		tbl.getTxn().blockId_tn_delete_metaLoc_batch.data[*blkId] =
  1789  			append(tbl.getTxn().blockId_tn_delete_metaLoc_batch.data[*blkId], copBat)
  1790  		tbl.getTxn().blockId_tn_delete_metaLoc_batch.RWMutex.Unlock()
  1791  
  1792  	case deletion.CNBlockOffset:
  1793  	case deletion.RawBatchOffset:
  1794  	case deletion.RawRowIdBatch:
  1795  		logutil.Infof("data return by remote pipeline\n")
  1796  		bat = tbl.getTxn().deleteBatch(bat, tbl.db.databaseId, tbl.tableId)
  1797  		if bat.RowCount() == 0 {
  1798  			return nil
  1799  		}
  1800  		tbl.writeTnPartition(tbl.getTxn().proc.Ctx, bat)
  1801  	default:
  1802  		tbl.getTxn().hasS3Op.Store(true)
  1803  		panic(moerr.NewInternalErrorNoCtx("Unsupport type for table delete %d", typ))
  1804  	}
  1805  	return nil
  1806  }
  1807  
  1808  func (tbl *txnTable) ensureSeqnumsAndTypesExpectRowid() {
  1809  	if tbl.seqnums != nil && tbl.typs != nil {
  1810  		return
  1811  	}
  1812  	n := len(tbl.tableDef.Cols) - 1
  1813  	idxs := make([]uint16, 0, n)
  1814  	typs := make([]types.Type, 0, n)
  1815  	for i := 0; i < len(tbl.tableDef.Cols)-1; i++ {
  1816  		col := tbl.tableDef.Cols[i]
  1817  		idxs = append(idxs, uint16(col.Seqnum))
  1818  		typs = append(typs, vector.ProtoTypeToType(&col.Typ))
  1819  	}
  1820  	tbl.seqnums = idxs
  1821  	tbl.typs = typs
  1822  }
  1823  
  1824  // TODO:: do prefetch read and parallel compaction
  1825  func (tbl *txnTable) compaction(
  1826  	compactedBlks map[objectio.ObjectLocation][]int64) ([]objectio.BlockInfo, []objectio.ObjectStats, error) {
  1827  	s3writer := &colexec.S3Writer{}
  1828  	s3writer.SetTableName(tbl.tableName)
  1829  	s3writer.SetSchemaVer(tbl.version)
  1830  	_, err := s3writer.GenerateWriter(tbl.getTxn().proc)
  1831  	if err != nil {
  1832  		return nil, nil, err
  1833  	}
  1834  	tbl.ensureSeqnumsAndTypesExpectRowid()
  1835  	s3writer.SetSeqnums(tbl.seqnums)
  1836  
  1837  	for blkmetaloc, deletes := range compactedBlks {
  1838  		//blk.MetaLocation()
  1839  		bat, e := blockio.BlockCompactionRead(
  1840  			tbl.getTxn().proc.Ctx,
  1841  			blkmetaloc[:],
  1842  			deletes,
  1843  			tbl.seqnums,
  1844  			tbl.typs,
  1845  			tbl.getTxn().engine.fs,
  1846  			tbl.getTxn().proc.GetMPool())
  1847  		if e != nil {
  1848  			return nil, nil, e
  1849  		}
  1850  		if bat.RowCount() == 0 {
  1851  			continue
  1852  		}
  1853  		s3writer.WriteBlock(bat)
  1854  		bat.Clean(tbl.getTxn().proc.GetMPool())
  1855  
  1856  	}
  1857  	createdBlks, stats, err := s3writer.WriteEndBlocks(tbl.getTxn().proc)
  1858  	if err != nil {
  1859  		return nil, nil, err
  1860  	}
  1861  	return createdBlks, stats, nil
  1862  }
  1863  
  1864  func (tbl *txnTable) Delete(ctx context.Context, bat *batch.Batch, name string) error {
  1865  	if tbl.db.op.IsSnapOp() {
  1866  		return moerr.NewInternalErrorNoCtx("delete operation is not allowed in snapshot transaction")
  1867  	}
  1868  	//for S3 delete
  1869  	if name != catalog.Row_ID {
  1870  		return tbl.EnhanceDelete(bat, name)
  1871  	}
  1872  	bat = tbl.getTxn().deleteBatch(bat, tbl.db.databaseId, tbl.tableId)
  1873  	if bat.RowCount() == 0 {
  1874  		return nil
  1875  	}
  1876  	return tbl.writeTnPartition(ctx, bat)
  1877  }
  1878  
  1879  func (tbl *txnTable) writeTnPartition(_ context.Context, bat *batch.Batch) error {
  1880  	ibat, err := util.CopyBatch(bat, tbl.getTxn().proc)
  1881  	if err != nil {
  1882  		return err
  1883  	}
  1884  	if err := tbl.getTxn().WriteBatch(DELETE, tbl.accountId, tbl.db.databaseId, tbl.tableId,
  1885  		tbl.db.databaseName, tbl.tableName, ibat, tbl.getTxn().tnStores[0], tbl.primaryIdx, false, false); err != nil {
  1886  		ibat.Clean(tbl.getTxn().proc.Mp())
  1887  		return err
  1888  	}
  1889  	return nil
  1890  }
  1891  
  1892  func (tbl *txnTable) AddTableDef(ctx context.Context, def engine.TableDef) error {
  1893  	return nil
  1894  }
  1895  
  1896  func (tbl *txnTable) DelTableDef(ctx context.Context, def engine.TableDef) error {
  1897  	return nil
  1898  }
  1899  
  1900  func (tbl *txnTable) GetTableID(ctx context.Context) uint64 {
  1901  	return tbl.tableId
  1902  }
  1903  
  1904  // GetTableName implements the engine.Relation interface.
  1905  func (tbl *txnTable) GetTableName() string {
  1906  	return tbl.tableName
  1907  }
  1908  
  1909  func (tbl *txnTable) GetDBID(ctx context.Context) uint64 {
  1910  	return tbl.db.databaseId
  1911  }
  1912  
  1913  func (tbl *txnTable) NewReader(
  1914  	ctx context.Context, num int, expr *plan.Expr, ranges []byte, orderedScan bool,
  1915  ) ([]engine.Reader, error) {
  1916  	pkFilter := tbl.tryExtractPKFilter(expr)
  1917  	blkArray := objectio.BlockInfoSlice(ranges)
  1918  	if pkFilter.isNull || plan2.IsFalseExpr(expr) {
  1919  		return []engine.Reader{new(emptyReader)}, nil
  1920  	}
  1921  	if blkArray.Len() == 0 {
  1922  		return tbl.newMergeReader(ctx, num, expr, pkFilter, nil)
  1923  	}
  1924  	if blkArray.Len() == 1 && engine.IsMemtable(blkArray.GetBytes(0)) {
  1925  		return tbl.newMergeReader(ctx, num, expr, pkFilter, nil)
  1926  	}
  1927  	if blkArray.Len() > 1 && engine.IsMemtable(blkArray.GetBytes(0)) {
  1928  		rds := make([]engine.Reader, num)
  1929  		mrds := make([]mergeReader, num)
  1930  		blkArray = blkArray.Slice(1, blkArray.Len())
  1931  
  1932  		var dirtyBlks []*objectio.BlockInfo
  1933  		var cleanBlks []*objectio.BlockInfo
  1934  		for i := 0; i < blkArray.Len(); i++ {
  1935  			blkInfo := blkArray.Get(i)
  1936  			if blkInfo.CanRemote {
  1937  				cleanBlks = append(cleanBlks, blkInfo)
  1938  				continue
  1939  			}
  1940  			dirtyBlks = append(dirtyBlks, blkInfo)
  1941  		}
  1942  		rds0, err := tbl.newMergeReader(ctx, num, expr, pkFilter, dirtyBlks)
  1943  		if err != nil {
  1944  			return nil, err
  1945  		}
  1946  		for i, rd := range rds0 {
  1947  			mrds[i].rds = append(mrds[i].rds, rd)
  1948  		}
  1949  
  1950  		if len(cleanBlks) > 0 {
  1951  			rds0, err = tbl.newBlockReader(ctx, num, expr, cleanBlks, tbl.proc.Load(), orderedScan)
  1952  			if err != nil {
  1953  				return nil, err
  1954  			}
  1955  		}
  1956  		for i, rd := range rds0 {
  1957  			mrds[i].rds = append(mrds[i].rds, rd)
  1958  		}
  1959  
  1960  		for i := range rds {
  1961  			rds[i] = &mrds[i]
  1962  		}
  1963  		return rds, nil
  1964  	}
  1965  	blkInfos := make([]*objectio.BlockInfo, 0, len(blkArray))
  1966  	for i := 0; i < blkArray.Len(); i++ {
  1967  		blkInfos = append(blkInfos, blkArray.Get(i))
  1968  	}
  1969  	return tbl.newBlockReader(ctx, num, expr, blkInfos, tbl.proc.Load(), orderedScan)
  1970  }
  1971  
  1972  func (tbl *txnTable) tryExtractPKFilter(expr *plan.Expr) (retPKFilter PKFilter) {
  1973  	pk := tbl.tableDef.Pkey
  1974  	if pk != nil && expr != nil {
  1975  		if pk.CompPkeyCol != nil {
  1976  			pkVals := make([]*plan.Literal, len(pk.Names))
  1977  			_, hasNull := getCompositPKVals(expr, pk.Names, pkVals, tbl.proc.Load())
  1978  			if hasNull {
  1979  				retPKFilter.SetNull()
  1980  				return
  1981  			}
  1982  			cnt := getValidCompositePKCnt(pkVals)
  1983  			if cnt != 0 {
  1984  				var packer *types.Packer
  1985  				put := tbl.getTxn().engine.packerPool.Get(&packer)
  1986  				for i := 0; i < cnt; i++ {
  1987  					serialTupleByConstExpr(pkVals[i], packer)
  1988  				}
  1989  				v := packer.Bytes()
  1990  				packer.Reset()
  1991  				pkValue := logtailreplay.EncodePrimaryKey(v, packer)
  1992  				// TODO: hack: remove the last comma, need to fix this in the future
  1993  				pkValue = pkValue[0 : len(pkValue)-1]
  1994  				put.Put()
  1995  				if cnt == len(pk.Names) {
  1996  					retPKFilter.SetFullData(function.EQUAL, false, pkValue)
  1997  				} else {
  1998  					retPKFilter.SetFullData(function.PREFIX_EQ, false, pkValue)
  1999  				}
  2000  			}
  2001  		} else {
  2002  			pkColumn := tbl.tableDef.Cols[tbl.primaryIdx]
  2003  			retPKFilter = getPKFilterByExpr(expr, pkColumn.Name, types.T(pkColumn.Typ.Id), tbl.proc.Load())
  2004  			if retPKFilter.isNull || !retPKFilter.isValid {
  2005  				return
  2006  			}
  2007  
  2008  			if !retPKFilter.isVec {
  2009  				var packer *types.Packer
  2010  				put := tbl.getTxn().engine.packerPool.Get(&packer)
  2011  				val := logtailreplay.EncodePrimaryKey(retPKFilter.val, packer)
  2012  				put.Put()
  2013  				if retPKFilter.op == function.EQUAL {
  2014  					retPKFilter.SetFullData(function.EQUAL, false, val)
  2015  				} else {
  2016  					// TODO: hack: remove the last comma, need to fix this in the future
  2017  					// serial_full(secondary_index, primary_key|fake_pk) => varchar
  2018  					// prefix_eq expression only has the prefix(secondary index) in it.
  2019  					// there will have an extra zero after the `encodeStringType` done
  2020  					// this will violate the rule of prefix_eq, so remove this redundant zero here.
  2021  					//
  2022  					val = val[0 : len(val)-1]
  2023  					retPKFilter.SetFullData(function.PREFIX_EQ, false, val)
  2024  				}
  2025  			}
  2026  
  2027  		}
  2028  		return
  2029  	}
  2030  	return
  2031  }
  2032  
  2033  func (tbl *txnTable) newMergeReader(
  2034  	ctx context.Context,
  2035  	num int,
  2036  	expr *plan.Expr,
  2037  	pkFilter PKFilter,
  2038  	dirtyBlks []*objectio.BlockInfo,
  2039  ) ([]engine.Reader, error) {
  2040  	rds := make([]engine.Reader, num)
  2041  	mrds := make([]mergeReader, num)
  2042  	rds0, err := tbl.newReader(
  2043  		ctx,
  2044  		num,
  2045  		pkFilter,
  2046  		expr,
  2047  		dirtyBlks)
  2048  	if err != nil {
  2049  		return nil, err
  2050  	}
  2051  	mrds[0].rds = append(mrds[0].rds, rds0...)
  2052  
  2053  	for i := range rds {
  2054  		rds[i] = &mrds[i]
  2055  	}
  2056  
  2057  	return rds, nil
  2058  }
  2059  
  2060  func (tbl *txnTable) newBlockReader(
  2061  	ctx context.Context,
  2062  	num int,
  2063  	expr *plan.Expr,
  2064  	blkInfos []*objectio.BlockInfo,
  2065  	proc *process.Process,
  2066  	orderedScan bool) ([]engine.Reader, error) {
  2067  	rds := make([]engine.Reader, num)
  2068  	ts := tbl.db.op.SnapshotTS()
  2069  	tableDef := tbl.GetTableDef(ctx)
  2070  
  2071  	if len(blkInfos) < num || len(blkInfos) == 1 {
  2072  		for i, blk := range blkInfos {
  2073  			rds[i] = newBlockReader(
  2074  				ctx,
  2075  				tableDef,
  2076  				ts,
  2077  				[]*objectio.BlockInfo{blk},
  2078  				expr,
  2079  				tbl.getTxn().engine.fs,
  2080  				proc,
  2081  			)
  2082  		}
  2083  		for j := len(blkInfos); j < num; j++ {
  2084  			rds[j] = &emptyReader{}
  2085  		}
  2086  		return rds, nil
  2087  	}
  2088  
  2089  	fs, err := fileservice.Get[fileservice.FileService](
  2090  		tbl.getTxn().engine.fs,
  2091  		defines.SharedFileServiceName)
  2092  	if err != nil {
  2093  		return nil, err
  2094  	}
  2095  
  2096  	if orderedScan {
  2097  		if num != 1 {
  2098  			panic("ordered scan must run in only one parallel")
  2099  		}
  2100  		rd := newBlockReader(ctx, tableDef, ts, blkInfos, expr, fs, proc)
  2101  		rd.dontPrefetch = true
  2102  		return []engine.Reader{rd}, nil
  2103  	}
  2104  
  2105  	infos, steps := groupBlocksToObjects(blkInfos, num)
  2106  	blockReaders := newBlockReaders(
  2107  		ctx,
  2108  		fs,
  2109  		tableDef,
  2110  		ts,
  2111  		num,
  2112  		expr,
  2113  		proc)
  2114  	distributeBlocksToBlockReaders(blockReaders, num, len(blkInfos), infos, steps)
  2115  	for i := 0; i < num; i++ {
  2116  		rds[i] = blockReaders[i]
  2117  	}
  2118  	return rds, nil
  2119  }
  2120  
  2121  func (tbl *txnTable) tryConstructPrimaryKeyIndexIter(
  2122  	ts timestamp.Timestamp,
  2123  	pkFilter PKFilter,
  2124  	expr *plan.Expr,
  2125  	state *logtailreplay.PartitionState,
  2126  ) (iter logtailreplay.RowsIter, newPkVal []byte) {
  2127  	if !pkFilter.isValid {
  2128  		return
  2129  	}
  2130  
  2131  	switch pkFilter.op {
  2132  	case function.EQUAL, function.PREFIX_EQ:
  2133  		newPkVal = pkFilter.data
  2134  		iter = state.NewPrimaryKeyIter(
  2135  			types.TimestampToTS(ts),
  2136  			logtailreplay.Prefix(pkFilter.data),
  2137  		)
  2138  	case function.IN:
  2139  		var encodes [][]byte
  2140  		vec := vector.NewVec(types.T_any.ToType())
  2141  		vec.UnmarshalBinary(pkFilter.data)
  2142  
  2143  		// may be it's better to iterate rows instead.
  2144  		if vec.Length() > 128 {
  2145  			return
  2146  		}
  2147  
  2148  		var packer *types.Packer
  2149  		put := tbl.getTxn().engine.packerPool.Get(&packer)
  2150  
  2151  		processed := false
  2152  		// case 1: serial_full(secondary_index, primary_key) ==> val, a in (val)
  2153  		if vec.Length() == 1 {
  2154  			exprLit := rule.GetConstantValue(vec, true, 0)
  2155  			if exprLit != nil && !exprLit.Isnull {
  2156  				canEval, val := evalLiteralExpr(exprLit, vec.GetType().Oid)
  2157  				if canEval {
  2158  					logtailreplay.EncodePrimaryKey(val, packer)
  2159  					newPkVal = packer.Bytes()
  2160  					encodes = append(encodes, newPkVal)
  2161  					processed = true
  2162  				}
  2163  			}
  2164  		}
  2165  
  2166  		if !processed {
  2167  			encodes = logtailreplay.EncodePrimaryKeyVector(vec, packer)
  2168  		}
  2169  
  2170  		put.Put()
  2171  
  2172  		iter = state.NewPrimaryKeyIter(
  2173  			types.TimestampToTS(ts),
  2174  			logtailreplay.ExactIn(encodes),
  2175  		)
  2176  	}
  2177  
  2178  	return iter, newPkVal
  2179  }
  2180  
  2181  func (tbl *txnTable) newReader(
  2182  	ctx context.Context,
  2183  	readerNumber int,
  2184  	pkFilter PKFilter,
  2185  	expr *plan.Expr,
  2186  	dirtyBlks []*objectio.BlockInfo,
  2187  ) ([]engine.Reader, error) {
  2188  	txn := tbl.getTxn()
  2189  	ts := txn.op.SnapshotTS()
  2190  	fs := txn.engine.fs
  2191  	state, err := tbl.getPartitionState(ctx)
  2192  	if err != nil {
  2193  		return nil, err
  2194  	}
  2195  	readers := make([]engine.Reader, readerNumber)
  2196  
  2197  	seqnumMp := make(map[string]int)
  2198  	for _, coldef := range tbl.tableDef.Cols {
  2199  		seqnumMp[coldef.Name] = int(coldef.Seqnum)
  2200  	}
  2201  
  2202  	mp := make(map[string]types.Type)
  2203  	mp[catalog.Row_ID] = types.New(types.T_Rowid, 0, 0)
  2204  	//FIXME::why did get type from the engine.AttributeDef,instead of plan.TableDef.Cols
  2205  	for _, def := range tbl.defs {
  2206  		attr, ok := def.(*engine.AttributeDef)
  2207  		if !ok {
  2208  			continue
  2209  		}
  2210  		mp[attr.Attr.Name] = attr.Attr.Type
  2211  	}
  2212  
  2213  	var (
  2214  		pkVal []byte
  2215  		iter  logtailreplay.RowsIter
  2216  	)
  2217  
  2218  	iter, pkVal = tbl.tryConstructPrimaryKeyIndexIter(ts, pkFilter, expr, state)
  2219  	if iter == nil {
  2220  		iter = state.NewRowsIter(
  2221  			types.TimestampToTS(ts),
  2222  			nil,
  2223  			false,
  2224  		)
  2225  	}
  2226  
  2227  	partReader := &PartitionReader{
  2228  		table:    tbl,
  2229  		iter:     iter,
  2230  		seqnumMp: seqnumMp,
  2231  		typsMap:  mp,
  2232  	}
  2233  
  2234  	//tbl.Lock()
  2235  	proc := tbl.proc.Load()
  2236  	//tbl.Unlock()
  2237  
  2238  	readers[0] = partReader
  2239  
  2240  	if readerNumber == 1 {
  2241  		for i := range dirtyBlks {
  2242  			readers = append(
  2243  				readers,
  2244  				newBlockMergeReader(
  2245  					ctx,
  2246  					tbl,
  2247  					pkVal,
  2248  					ts,
  2249  					[]*objectio.BlockInfo{dirtyBlks[i]},
  2250  					expr,
  2251  					fs,
  2252  					proc,
  2253  				),
  2254  			)
  2255  		}
  2256  		return []engine.Reader{&mergeReader{readers}}, nil
  2257  	}
  2258  
  2259  	if len(dirtyBlks) < readerNumber-1 {
  2260  		for i := range dirtyBlks {
  2261  			readers[i+1] = newBlockMergeReader(
  2262  				ctx,
  2263  				tbl,
  2264  				pkVal,
  2265  				ts,
  2266  				[]*objectio.BlockInfo{dirtyBlks[i]},
  2267  				expr,
  2268  				fs,
  2269  				proc,
  2270  			)
  2271  		}
  2272  		for j := len(dirtyBlks) + 1; j < readerNumber; j++ {
  2273  			readers[j] = &emptyReader{}
  2274  		}
  2275  		return readers, nil
  2276  	}
  2277  	//create readerNumber-1 blockReaders
  2278  	blockReaders := newBlockReaders(
  2279  		ctx,
  2280  		fs,
  2281  		tbl.tableDef,
  2282  		ts,
  2283  		readerNumber-1,
  2284  		expr,
  2285  		proc)
  2286  	objInfos, steps := groupBlocksToObjects(dirtyBlks, readerNumber-1)
  2287  	blockReaders = distributeBlocksToBlockReaders(
  2288  		blockReaders,
  2289  		readerNumber-1,
  2290  		len(dirtyBlks),
  2291  		objInfos,
  2292  		steps)
  2293  	for i := range blockReaders {
  2294  		bmr := &blockMergeReader{
  2295  			blockReader: blockReaders[i],
  2296  			table:       tbl,
  2297  			pkVal:       pkVal,
  2298  			deletaLocs:  make(map[string][]objectio.Location),
  2299  		}
  2300  		readers[i+1] = bmr
  2301  	}
  2302  	return readers, nil
  2303  }
  2304  
  2305  func (tbl *txnTable) getPartitionState(
  2306  	ctx context.Context,
  2307  ) (*logtailreplay.PartitionState, error) {
  2308  	if !tbl.db.op.IsSnapOp() {
  2309  		if tbl._partState.Load() == nil {
  2310  			if err := tbl.updateLogtail(ctx); err != nil {
  2311  				return nil, err
  2312  			}
  2313  			tbl._partState.Store(tbl.getTxn().engine.
  2314  				getOrCreateLatestPart(tbl.db.databaseId, tbl.tableId).Snapshot())
  2315  		}
  2316  		return tbl._partState.Load(), nil
  2317  	}
  2318  
  2319  	// for snapshot txnOp
  2320  	if tbl._partState.Load() == nil {
  2321  		p, err := tbl.getTxn().engine.getOrCreateSnapPart(
  2322  			ctx,
  2323  			tbl,
  2324  			types.TimestampToTS(tbl.db.op.Txn().SnapshotTS))
  2325  		if err != nil {
  2326  			return nil, err
  2327  		}
  2328  		tbl._partState.Store(p.Snapshot())
  2329  	}
  2330  	return tbl._partState.Load(), nil
  2331  }
  2332  
  2333  func (tbl *txnTable) updateLogtail(ctx context.Context) (err error) {
  2334  	defer func() {
  2335  		if err == nil {
  2336  			tbl.getTxn().engine.globalStats.notifyLogtailUpdate(tbl.tableId)
  2337  			tbl.logtailUpdated.Store(true)
  2338  		}
  2339  	}()
  2340  	// if the logtail is updated, skip
  2341  	if tbl.logtailUpdated.Load() {
  2342  		return
  2343  	}
  2344  
  2345  	// if the table is created in this txn, skip
  2346  	accountId, err := defines.GetAccountId(ctx)
  2347  	if err != nil {
  2348  		return err
  2349  	}
  2350  	if _, created := tbl.getTxn().createMap.Load(
  2351  		genTableKey(accountId, tbl.tableName, tbl.db.databaseId)); created {
  2352  		return
  2353  	}
  2354  
  2355  	tableId := tbl.tableId
  2356  	/*
  2357  		if the table is truncated once or more than once,
  2358  		it is suitable to use the old table id to sync logtail.
  2359  
  2360  		CORNER CASE 1:
  2361  		create table t1(a int);
  2362  		begin;
  2363  		truncate t1; //table id changed. there is no new table id in DN.
  2364  		select count(*) from t1; // sync logtail for the new id failed.
  2365  
  2366  		CORNER CASE 2:
  2367  		create table t1(a int);
  2368  		begin;
  2369  		select count(*) from t1; // sync logtail for the old succeeded.
  2370  		truncate t1; //table id changed. there is no new table id in DN.
  2371  		select count(*) from t1; // not sync logtail this time.
  2372  
  2373  		CORNER CASE 3:
  2374  		create table t1(a int);
  2375  		begin;
  2376  		truncate t1; //table id changed. there is no new table id in DN.
  2377  		truncate t1; //table id changed. there is no new table id in DN.
  2378  		select count(*) from t1; // sync logtail for the new id failed.
  2379  	*/
  2380  	if tbl.oldTableId != 0 {
  2381  		tableId = tbl.oldTableId
  2382  	}
  2383  
  2384  	if err = tbl.getTxn().engine.UpdateOfPush(ctx, tbl.db.databaseId, tableId,
  2385  		tbl.db.op.SnapshotTS()); err != nil {
  2386  		return
  2387  	}
  2388  	if _, err = tbl.getTxn().engine.lazyLoadLatestCkp(ctx, tbl); err != nil {
  2389  		return
  2390  	}
  2391  
  2392  	return nil
  2393  }
  2394  
  2395  func (tbl *txnTable) PKPersistedBetween(
  2396  	p *logtailreplay.PartitionState,
  2397  	from types.TS,
  2398  	to types.TS,
  2399  	keys *vector.Vector,
  2400  ) (bool, error) {
  2401  
  2402  	ctx := tbl.proc.Load().Ctx
  2403  	fs := tbl.getTxn().engine.fs
  2404  	primaryIdx := tbl.primaryIdx
  2405  
  2406  	var (
  2407  		meta objectio.ObjectDataMeta
  2408  		bf   objectio.BloomFilter
  2409  	)
  2410  
  2411  	candidateBlks := make(map[types.Blockid]*objectio.BlockInfo)
  2412  
  2413  	//only check data objects.
  2414  	delObjs, cObjs := p.GetChangedObjsBetween(from.Next(), types.MaxTs())
  2415  	isFakePK := tbl.GetTableDef(ctx).Pkey.PkeyColName == catalog.FakePrimaryKeyColName
  2416  
  2417  	if err := ForeachCommittedObjects(cObjs, delObjs, p,
  2418  		func(obj logtailreplay.ObjectInfo) (err2 error) {
  2419  			var zmCkecked bool
  2420  			if !isFakePK {
  2421  				// if the object info contains a pk zonemap, fast-check with the zonemap
  2422  				if !obj.ZMIsEmpty() {
  2423  					if !obj.SortKeyZoneMap().AnyIn(keys) {
  2424  						return
  2425  					}
  2426  					zmCkecked = true
  2427  				}
  2428  			}
  2429  
  2430  			var objMeta objectio.ObjectMeta
  2431  			location := obj.Location()
  2432  
  2433  			// load object metadata
  2434  			if objMeta, err2 = objectio.FastLoadObjectMeta(
  2435  				ctx, &location, false, fs,
  2436  			); err2 != nil {
  2437  				return
  2438  			}
  2439  
  2440  			// reset bloom filter to nil for each object
  2441  			meta = objMeta.MustDataMeta()
  2442  
  2443  			// check whether the object is skipped by zone map
  2444  			// If object zone map doesn't contains the pk value, we need to check bloom filter
  2445  			if !zmCkecked {
  2446  				if !meta.MustGetColumn(uint16(primaryIdx)).ZoneMap().AnyIn(keys) {
  2447  					return
  2448  				}
  2449  			}
  2450  
  2451  			bf = nil
  2452  			//fake pk has no bf
  2453  			if !isFakePK {
  2454  				if bf, err2 = objectio.LoadBFWithMeta(
  2455  					ctx, meta, location, fs,
  2456  				); err2 != nil {
  2457  					return
  2458  				}
  2459  			}
  2460  
  2461  			ForeachBlkInObjStatsList(false, meta,
  2462  				func(blk objectio.BlockInfo, blkMeta objectio.BlockObject) bool {
  2463  					if !blkMeta.IsEmpty() &&
  2464  						!blkMeta.MustGetColumn(uint16(primaryIdx)).ZoneMap().AnyIn(keys) {
  2465  						return true
  2466  					}
  2467  					//fake pk has no bf
  2468  					if !isFakePK {
  2469  						blkBf := bf.GetBloomFilter(uint32(blk.BlockID.Sequence()))
  2470  						blkBfIdx := index.NewEmptyBinaryFuseFilter()
  2471  						if err2 = index.DecodeBloomFilter(blkBfIdx, blkBf); err2 != nil {
  2472  							return false
  2473  						}
  2474  						var exist bool
  2475  						lowerBound, upperBound := blkMeta.MustGetColumn(uint16(primaryIdx)).ZoneMap().SubVecIn(keys)
  2476  						if exist = blkBfIdx.MayContainsAny(keys, lowerBound, upperBound); !exist {
  2477  							return true
  2478  						}
  2479  					}
  2480  
  2481  					blk.Sorted = obj.Sorted
  2482  					blk.EntryState = obj.EntryState
  2483  					blk.CommitTs = obj.CommitTS
  2484  					if obj.HasDeltaLoc {
  2485  						deltaLoc, commitTs, ok := p.GetBockDeltaLoc(blk.BlockID)
  2486  						if ok {
  2487  							blk.DeltaLoc = deltaLoc
  2488  							blk.CommitTs = commitTs
  2489  						}
  2490  					}
  2491  					blk.PartitionNum = -1
  2492  					candidateBlks[blk.BlockID] = &blk
  2493  					return true
  2494  				}, obj.ObjectStats)
  2495  
  2496  			return
  2497  		}); err != nil {
  2498  		return true, err
  2499  	}
  2500  
  2501  	var filter blockio.ReadFilter
  2502  	buildFilter := func() blockio.ReadFilter {
  2503  		//keys must be sorted.
  2504  		keys.InplaceSort()
  2505  		bytes, _ := keys.MarshalBinary()
  2506  		colExpr := newColumnExpr(0, plan2.MakePlan2Type(keys.GetType()), "pk")
  2507  		inExpr := plan2.MakeInExpr(
  2508  			tbl.proc.Load().Ctx,
  2509  			colExpr,
  2510  			int32(keys.Length()),
  2511  			bytes,
  2512  			false)
  2513  
  2514  		_, _, filter := getNonCompositePKSearchFuncByExpr(
  2515  			inExpr,
  2516  			"pk",
  2517  			tbl.proc.Load())
  2518  		return filter
  2519  	}
  2520  
  2521  	var unsortedFilter blockio.ReadFilter
  2522  	buildUnsortedFilter := func() blockio.ReadFilter {
  2523  		return getNonSortedPKSearchFuncByPKVec(keys)
  2524  	}
  2525  
  2526  	//read block ,check if keys exist in the block.
  2527  	pkDef := tbl.tableDef.Cols[tbl.primaryIdx]
  2528  	pkSeq := pkDef.Seqnum
  2529  	pkType := types.T(pkDef.Typ.Id).ToType()
  2530  	for _, blk := range candidateBlks {
  2531  		bat, release, err := blockio.LoadColumns(
  2532  			ctx,
  2533  			[]uint16{uint16(pkSeq)},
  2534  			[]types.Type{pkType},
  2535  			fs,
  2536  			blk.MetaLocation(),
  2537  			tbl.proc.Load().GetMPool(),
  2538  			fileservice.Policy(0),
  2539  		)
  2540  		if err != nil {
  2541  			return true, err
  2542  		}
  2543  		defer release()
  2544  
  2545  		if !blk.Sorted {
  2546  			if unsortedFilter == nil {
  2547  				unsortedFilter = buildUnsortedFilter()
  2548  			}
  2549  			sels := unsortedFilter(bat.Vecs)
  2550  			if len(sels) > 0 {
  2551  				return true, nil
  2552  			}
  2553  			continue
  2554  		}
  2555  
  2556  		//for sorted block, we can use binary search to find the keys.
  2557  		if filter == nil {
  2558  			filter = buildFilter()
  2559  		}
  2560  		sels := filter(bat.Vecs)
  2561  		if len(sels) > 0 {
  2562  			return true, nil
  2563  		}
  2564  	}
  2565  	return false, nil
  2566  }
  2567  
  2568  func (tbl *txnTable) PrimaryKeysMayBeModified(
  2569  	ctx context.Context,
  2570  	from types.TS,
  2571  	to types.TS,
  2572  	keysVector *vector.Vector) (bool, error) {
  2573  	if tbl.db.op.IsSnapOp() {
  2574  		return false,
  2575  			moerr.NewInternalErrorNoCtx("primary key modification is not allowed in snapshot transaction")
  2576  	}
  2577  	part, err := tbl.getTxn().engine.lazyLoadLatestCkp(ctx, tbl)
  2578  	if err != nil {
  2579  		return false, err
  2580  	}
  2581  
  2582  	snap := part.Snapshot()
  2583  	var packer *types.Packer
  2584  	put := tbl.getTxn().engine.packerPool.Get(&packer)
  2585  	defer put.Put()
  2586  	packer.Reset()
  2587  
  2588  	keys := logtailreplay.EncodePrimaryKeyVector(keysVector, packer)
  2589  	exist, flushed := snap.PKExistInMemBetween(from, to, keys)
  2590  	if exist {
  2591  		return true, nil
  2592  	}
  2593  	if !flushed {
  2594  		return false, nil
  2595  	}
  2596  	//for mo_tables, mo_database, mo_columns, pk always exist in memory.
  2597  	if tbl.tableName == catalog.MO_DATABASE ||
  2598  		tbl.tableName == catalog.MO_TABLES ||
  2599  		tbl.tableName == catalog.MO_COLUMNS {
  2600  		logutil.Warnf("mo table:%s always exist in memory", tbl.tableName)
  2601  		return true, nil
  2602  	}
  2603  	//need check pk whether exist on S3 block.
  2604  	return tbl.PKPersistedBetween(
  2605  		snap,
  2606  		from,
  2607  		to,
  2608  		keysVector)
  2609  }
  2610  
  2611  // TODO::refactor in next PR
  2612  func (tbl *txnTable) transferDeletes(
  2613  	ctx context.Context,
  2614  	state *logtailreplay.PartitionState,
  2615  	deleteObjs,
  2616  	createObjs map[objectio.ObjectNameShort]struct{}) error {
  2617  	var blks []objectio.BlockInfo
  2618  
  2619  	{
  2620  		fs, err := fileservice.Get[fileservice.FileService](
  2621  			tbl.proc.Load().FileService,
  2622  			defines.SharedFileServiceName)
  2623  		if err != nil {
  2624  			return err
  2625  		}
  2626  		var objDataMeta objectio.ObjectDataMeta
  2627  		var objMeta objectio.ObjectMeta
  2628  		for name := range createObjs {
  2629  			if obj, ok := state.GetObject(name); ok {
  2630  				location := obj.Location()
  2631  				if objMeta, err = objectio.FastLoadObjectMeta(
  2632  					ctx,
  2633  					&location,
  2634  					false,
  2635  					fs); err != nil {
  2636  					return err
  2637  				}
  2638  				objDataMeta = objMeta.MustDataMeta()
  2639  				blkCnt := objDataMeta.BlockCount()
  2640  				for i := 0; i < int(blkCnt); i++ {
  2641  					blkMeta := objDataMeta.GetBlockMeta(uint32(i))
  2642  					bid := *blkMeta.GetBlockID(obj.Location().Name())
  2643  					metaLoc := blockio.EncodeLocation(
  2644  						obj.Location().Name(),
  2645  						obj.Location().Extent(),
  2646  						blkMeta.GetRows(),
  2647  						blkMeta.GetID(),
  2648  					)
  2649  					blkInfo := objectio.BlockInfo{
  2650  						BlockID:    bid,
  2651  						EntryState: obj.EntryState,
  2652  						Sorted:     obj.Sorted,
  2653  						MetaLoc:    *(*[objectio.LocationLen]byte)(unsafe.Pointer(&metaLoc[0])),
  2654  						CommitTs:   obj.CommitTS,
  2655  						SegmentID:  *obj.ObjectShortName().Segmentid(),
  2656  					}
  2657  					if obj.HasDeltaLoc {
  2658  						deltaLoc, commitTs, ok := state.GetBockDeltaLoc(blkInfo.BlockID)
  2659  						if ok {
  2660  							blkInfo.DeltaLoc = deltaLoc
  2661  							blkInfo.CommitTs = commitTs
  2662  						}
  2663  					}
  2664  					blks = append(blks, blkInfo)
  2665  				}
  2666  			}
  2667  		}
  2668  	}
  2669  
  2670  	for _, entry := range tbl.getTxn().writes {
  2671  		if entry.isGeneratedByTruncate() || entry.tableId != tbl.tableId {
  2672  			continue
  2673  		}
  2674  		if (entry.typ == DELETE || entry.typ == DELETE_TXN) && entry.fileName == "" {
  2675  			pkVec := entry.bat.GetVector(1)
  2676  			rowids := vector.MustFixedCol[types.Rowid](entry.bat.GetVector(0))
  2677  			beTransfered := 0
  2678  			toTransfer := 0
  2679  			for i, rowid := range rowids {
  2680  				blkid, _ := rowid.Decode()
  2681  				if _, ok := deleteObjs[*objectio.ShortName(&blkid)]; ok {
  2682  					toTransfer++
  2683  					newId, ok, err := tbl.readNewRowid(pkVec, i, blks)
  2684  					if err != nil {
  2685  						return err
  2686  					}
  2687  					if ok {
  2688  						newBlockID, _ := newId.Decode()
  2689  						trace.GetService().ApplyTransferRowID(
  2690  							tbl.db.op.Txn().ID,
  2691  							tbl.tableId,
  2692  							rowids[i][:],
  2693  							newId[:],
  2694  							blkid[:],
  2695  							newBlockID[:],
  2696  							pkVec,
  2697  							i)
  2698  						rowids[i] = newId
  2699  						beTransfered++
  2700  					}
  2701  				}
  2702  			}
  2703  			if beTransfered != toTransfer {
  2704  				return moerr.NewInternalErrorNoCtx("transfer deletes failed")
  2705  			}
  2706  		}
  2707  	}
  2708  	return nil
  2709  }
  2710  
  2711  func (tbl *txnTable) readNewRowid(vec *vector.Vector, row int,
  2712  	blks []objectio.BlockInfo) (types.Rowid, bool, error) {
  2713  	var auxIdCnt int32
  2714  	var typ plan.Type
  2715  	var rowid types.Rowid
  2716  	var objMeta objectio.ObjectMeta
  2717  
  2718  	columns := []uint16{objectio.SEQNUM_ROWID}
  2719  	colTypes := []types.Type{objectio.RowidType}
  2720  	tableDef := tbl.GetTableDef(context.TODO())
  2721  	for _, col := range tableDef.Cols {
  2722  		if col.Name == tableDef.Pkey.PkeyColName {
  2723  			typ = col.Typ
  2724  			columns = append(columns, uint16(col.Seqnum))
  2725  			colTypes = append(colTypes, types.T(col.Typ.Id).ToType())
  2726  		}
  2727  	}
  2728  	constExpr := getConstExpr(int32(vec.GetType().Oid),
  2729  		rule.GetConstantValue(vec, true, uint64(row)))
  2730  	filter, err := tbl.newPkFilter(newColumnExpr(1, typ, tableDef.Pkey.PkeyColName), constExpr)
  2731  	if err != nil {
  2732  		return rowid, false, err
  2733  	}
  2734  	columnMap := make(map[int]int)
  2735  	auxIdCnt += plan2.AssignAuxIdForExpr(filter, auxIdCnt)
  2736  	zms := make([]objectio.ZoneMap, auxIdCnt)
  2737  	vecs := make([]*vector.Vector, auxIdCnt)
  2738  	plan2.GetColumnMapByExprs([]*plan.Expr{filter}, tableDef, columnMap)
  2739  	objFilterMap := make(map[objectio.ObjectNameShort]bool)
  2740  	for _, blk := range blks {
  2741  		location := blk.MetaLocation()
  2742  		if hit, ok := objFilterMap[*location.ShortName()]; !ok {
  2743  			if objMeta, err = objectio.FastLoadObjectMeta(
  2744  				tbl.proc.Load().Ctx, &location, false, tbl.getTxn().engine.fs,
  2745  			); err != nil {
  2746  				return rowid, false, err
  2747  			}
  2748  			hit = colexec.EvaluateFilterByZoneMap(tbl.proc.Load().Ctx, tbl.proc.Load(), filter,
  2749  				objMeta.MustDataMeta(), columnMap, zms, vecs)
  2750  			objFilterMap[*location.ShortName()] = hit
  2751  			if !hit {
  2752  				continue
  2753  			}
  2754  		} else if !hit {
  2755  			continue
  2756  		}
  2757  		// eval filter expr on the block
  2758  		blkMeta := objMeta.MustDataMeta().GetBlockMeta(uint32(location.ID()))
  2759  		if !colexec.EvaluateFilterByZoneMap(tbl.proc.Load().Ctx, tbl.proc.Load(), filter,
  2760  			blkMeta, columnMap, zms, vecs) {
  2761  			continue
  2762  		}
  2763  		// rowid + pk
  2764  		bat, err := blockio.BlockRead(
  2765  			tbl.proc.Load().Ctx, &blk, nil, columns, colTypes,
  2766  			tbl.db.op.SnapshotTS(),
  2767  			nil, nil, nil,
  2768  			tbl.getTxn().engine.fs, tbl.proc.Load().Mp(), tbl.proc.Load(), fileservice.Policy(0),
  2769  		)
  2770  		if err != nil {
  2771  			return rowid, false, err
  2772  		}
  2773  		vec, err := colexec.EvalExpressionOnce(tbl.getTxn().proc, filter, []*batch.Batch{bat})
  2774  		if err != nil {
  2775  			return rowid, false, err
  2776  		}
  2777  		bs := vector.MustFixedCol[bool](vec)
  2778  		for i, b := range bs {
  2779  			if b {
  2780  				rowids := vector.MustFixedCol[types.Rowid](bat.Vecs[0])
  2781  				vec.Free(tbl.proc.Load().Mp())
  2782  				bat.Clean(tbl.proc.Load().Mp())
  2783  				return rowids[i], true, nil
  2784  			}
  2785  		}
  2786  		vec.Free(tbl.proc.Load().Mp())
  2787  		bat.Clean(tbl.proc.Load().Mp())
  2788  	}
  2789  	return rowid, false, nil
  2790  }
  2791  
  2792  func (tbl *txnTable) newPkFilter(pkExpr, constExpr *plan.Expr) (*plan.Expr, error) {
  2793  	return plan2.BindFuncExprImplByPlanExpr(tbl.proc.Load().Ctx, "=", []*plan.Expr{pkExpr, constExpr})
  2794  }
  2795  
  2796  func (tbl *txnTable) MergeObjects(ctx context.Context, objstats []objectio.ObjectStats, policyName string, targetObjSize uint32) (*api.MergeCommitEntry, error) {
  2797  	snapshot := types.TimestampToTS(tbl.getTxn().op.SnapshotTS())
  2798  	state, err := tbl.getPartitionState(ctx)
  2799  	if err != nil {
  2800  		return nil, err
  2801  	}
  2802  
  2803  	sortkeyPos := -1
  2804  	sortkeyIsPK := false
  2805  	if tbl.primaryIdx >= 0 && tbl.tableDef.Cols[tbl.primaryIdx].Name != catalog.FakePrimaryKeyColName {
  2806  		if tbl.clusterByIdx < 0 {
  2807  			sortkeyPos = tbl.primaryIdx
  2808  			sortkeyIsPK = true
  2809  		} else {
  2810  			panic(fmt.Sprintf("bad schema pk %v, ck %v", tbl.primaryIdx, tbl.clusterByIdx))
  2811  		}
  2812  	} else if tbl.clusterByIdx >= 0 {
  2813  		sortkeyPos = tbl.clusterByIdx
  2814  		sortkeyIsPK = false
  2815  	}
  2816  
  2817  	var objInfos []logtailreplay.ObjectInfo
  2818  	if len(objstats) != 0 {
  2819  		objInfos = make([]logtailreplay.ObjectInfo, 0, len(objstats))
  2820  		for _, objstat := range objstats {
  2821  			info, exist := state.GetObject(*objstat.ObjectShortName())
  2822  			if !exist || (!info.DeleteTime.IsEmpty() && info.DeleteTime.LessEq(&snapshot)) {
  2823  				logutil.Errorf("object not visible: %s", info.String())
  2824  				return nil, moerr.NewInternalErrorNoCtx("object %s not exist", objstat.ObjectName().String())
  2825  			}
  2826  			objInfos = append(objInfos, info)
  2827  		}
  2828  	} else {
  2829  		objInfos = make([]logtailreplay.ObjectInfo, 0)
  2830  		iter, err := state.NewObjectsIter(snapshot)
  2831  		if err != nil {
  2832  			return nil, err
  2833  		}
  2834  		for iter.Next() {
  2835  			obj := iter.Entry().ObjectInfo
  2836  			if obj.EntryState {
  2837  				continue
  2838  			}
  2839  			if sortkeyPos != -1 {
  2840  				sortKeyZM := obj.SortKeyZoneMap()
  2841  				if !sortKeyZM.IsInited() {
  2842  					continue
  2843  				}
  2844  			}
  2845  			objInfos = append(objInfos, obj)
  2846  		}
  2847  		if len(policyName) != 0 {
  2848  			if strings.HasPrefix(policyName, "small") {
  2849  				objInfos = logtailreplay.NewSmall(110 * common.Const1MBytes).Filter(objInfos)
  2850  			} else if strings.HasPrefix(policyName, "overlap") {
  2851  				if sortkeyPos != -1 {
  2852  					objInfos = logtailreplay.NewOverlap(100).Filter(objInfos)
  2853  				}
  2854  			} else {
  2855  				return nil, moerr.NewInvalidInput(ctx, "invalid merge policy name")
  2856  			}
  2857  		}
  2858  	}
  2859  
  2860  	if len(objInfos) < 2 {
  2861  		return nil, moerr.NewInternalErrorNoCtx("no matching objects")
  2862  	}
  2863  
  2864  	tbl.ensureSeqnumsAndTypesExpectRowid()
  2865  
  2866  	taskHost, err := newCNMergeTask(
  2867  		ctx, tbl, snapshot, state, // context
  2868  		sortkeyPos, sortkeyIsPK, // schema
  2869  		objInfos, // targets
  2870  		targetObjSize)
  2871  	if err != nil {
  2872  		return nil, err
  2873  	}
  2874  
  2875  	err = mergesort.DoMergeAndWrite(ctx, sortkeyPos, int(options.DefaultBlockMaxRows), taskHost)
  2876  	if err != nil {
  2877  		return nil, err
  2878  	}
  2879  
  2880  	if taskHost.DoTransfer() {
  2881  		return taskHost.commitEntry, nil
  2882  	}
  2883  
  2884  	// if transfer info is too large, write it down to s3
  2885  	if size := taskHost.commitEntry.Booking.ProtoSize(); size > 80*common.Const1MBytes {
  2886  		if err := dumpTransferInfo(ctx, taskHost); err != nil {
  2887  			return taskHost.commitEntry, err
  2888  		}
  2889  		idx := 0
  2890  		locstr := ""
  2891  		locations := taskHost.commitEntry.BookingLoc
  2892  		for ; idx < len(locations); idx += objectio.LocationLen {
  2893  			loc := objectio.Location(locations[idx : idx+objectio.LocationLen])
  2894  			locstr += loc.String() + ","
  2895  		}
  2896  		logutil.Infof("mergeblocks %v-%v on cn: write s3 transfer info (%v) %v",
  2897  			tbl.tableId, tbl.tableName, common.HumanReadableBytes(size), locstr)
  2898  	}
  2899  
  2900  	// commit this to tn
  2901  	return taskHost.commitEntry, nil
  2902  }
  2903  
  2904  func dumpTransferInfo(ctx context.Context, taskHost *cnMergeTask) (err error) {
  2905  	defer func() {
  2906  		if err != nil {
  2907  			idx := 0
  2908  			locations := taskHost.commitEntry.BookingLoc
  2909  			for ; idx < len(locations); idx += objectio.LocationLen {
  2910  				loc := objectio.Location(locations[idx : idx+objectio.LocationLen])
  2911  				taskHost.fs.Delete(ctx, loc.Name().String())
  2912  			}
  2913  		}
  2914  	}()
  2915  	data, err := taskHost.commitEntry.Booking.Marshal()
  2916  	if err != nil {
  2917  		return
  2918  	}
  2919  	taskHost.commitEntry.BookingLoc = make([]byte, 0, objectio.LocationLen)
  2920  	chunkSize := 1000 * mpool.MB
  2921  	for len(data) > 0 {
  2922  		var chunck []byte
  2923  		if len(data) > chunkSize {
  2924  			chunck = data[:chunkSize]
  2925  			data = data[chunkSize:]
  2926  		} else {
  2927  			chunck = data
  2928  			data = nil
  2929  		}
  2930  
  2931  		t := types.T_varchar.ToType()
  2932  		v, releasev := taskHost.GetVector(&t)
  2933  		vectorRowCnt := 1
  2934  		vector.AppendBytes(v, chunck, false, taskHost.GetMPool())
  2935  		name := objectio.BuildObjectName(objectio.NewSegmentid(), 0)
  2936  		var writer *blockio.BlockWriter
  2937  		writer, err = blockio.NewBlockWriterNew(taskHost.fs, name, 0, nil)
  2938  		if err != nil {
  2939  			releasev()
  2940  			return
  2941  		}
  2942  
  2943  		batch := batch.New(true, []string{"payload"})
  2944  		batch.SetRowCount(vectorRowCnt)
  2945  		batch.Vecs[0] = v
  2946  		_, err = writer.WriteTombstoneBatch(batch)
  2947  		if err != nil {
  2948  			releasev()
  2949  			return
  2950  		}
  2951  		var blocks []objectio.BlockObject
  2952  		blocks, _, err = writer.Sync(ctx)
  2953  		releasev()
  2954  		if err != nil {
  2955  			return
  2956  		}
  2957  		location := blockio.EncodeLocation(
  2958  			name,
  2959  			blocks[0].GetExtent(),
  2960  			uint32(vectorRowCnt),
  2961  			blocks[0].GetID())
  2962  		taskHost.commitEntry.BookingLoc = append(taskHost.commitEntry.BookingLoc, location...)
  2963  	}
  2964  
  2965  	taskHost.commitEntry.Booking = nil
  2966  	return
  2967  }