github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/tables/updates/mvcc.go (about)

     1  // Copyright 2021 Matrix Origin
     2  //
     3  // Licensed under the Apache License, Version 2.0 (the "License");
     4  // you may not use this file except in compliance with the License.
     5  // You may obtain a copy of the License at
     6  //
     7  //      http://www.apache.org/licenses/LICENSE-2.0
     8  //
     9  // Unless required by applicable law or agreed to in writing, software
    10  // distributed under the License is distributed on an "AS IS" BASIS,
    11  // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    12  // See the License for the specific language governing permissions and
    13  // limitations under the License.
    14  
    15  package updates
    16  
    17  import (
    18  	"context"
    19  	"fmt"
    20  	"sync"
    21  	"sync/atomic"
    22  	"unsafe"
    23  
    24  	"github.com/matrixorigin/matrixone/pkg/common/mpool"
    25  	"github.com/matrixorigin/matrixone/pkg/container/nulls"
    26  	"github.com/matrixorigin/matrixone/pkg/container/types"
    27  	"github.com/matrixorigin/matrixone/pkg/logutil"
    28  	"github.com/matrixorigin/matrixone/pkg/objectio"
    29  
    30  	pkgcatalog "github.com/matrixorigin/matrixone/pkg/catalog"
    31  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio"
    32  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog"
    33  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    34  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers"
    35  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/handle"
    36  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif"
    37  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase"
    38  )
    39  
    40  var (
    41  	AppendNodeApproxSize int
    42  	DeleteNodeApproxSize int
    43  
    44  	DeleteChainApproxSize int
    45  	MVCCHandleApproxSize  int
    46  )
    47  
    48  func init() {
    49  	txnNodeSize := int(unsafe.Sizeof(txnbase.TxnMVCCNode{}))
    50  	AppendNodeApproxSize = int(unsafe.Sizeof(AppendNode{})) + txnNodeSize
    51  	DeleteNodeApproxSize = int(unsafe.Sizeof(DeleteNode{})) + txnNodeSize
    52  
    53  	DeleteChainApproxSize = int(unsafe.Sizeof(DeleteChain{}))
    54  	MVCCHandleApproxSize = int(unsafe.Sizeof(MVCCHandle{}))
    55  }
    56  
    57  type AppendMVCCHandle struct {
    58  	*sync.RWMutex
    59  	meta           *catalog.ObjectEntry
    60  	appends        *txnbase.MVCCSlice[*AppendNode]
    61  	appendListener func(txnif.AppendNode) error
    62  }
    63  
    64  func NewAppendMVCCHandle(meta *catalog.ObjectEntry) *AppendMVCCHandle {
    65  	node := &AppendMVCCHandle{
    66  		RWMutex: meta.RWMutex,
    67  		meta:    meta,
    68  		appends: txnbase.NewMVCCSlice(NewEmptyAppendNode, CompareAppendNode),
    69  	}
    70  	return node
    71  }
    72  
    73  // ==========================================================
    74  // *************** All appends related APIs *****************
    75  // ==========================================================
    76  
    77  // NOTE: after this call all appends related APIs should not be called
    78  // ReleaseAppends release all append nodes.
    79  // it is only called when the appendable block is persisted and the
    80  // memory node is released
    81  func (n *AppendMVCCHandle) ReleaseAppends() {
    82  	n.Lock()
    83  	defer n.Unlock()
    84  	n.appends = nil
    85  }
    86  
    87  // only for internal usage
    88  // given a row, it returns the append node which contains the row
    89  func (n *AppendMVCCHandle) GetAppendNodeByRow(row uint32) (an *AppendNode) {
    90  	_, an = n.appends.SearchNodeByCompareFn(func(node *AppendNode) int {
    91  		if node.maxRow <= row {
    92  			return -1
    93  		}
    94  		if node.startRow > row {
    95  			return 1
    96  		}
    97  		return 0
    98  	})
    99  	return
   100  }
   101  
   102  // it collects all append nodes in the range [start, end]
   103  // minRow: is the min row
   104  // maxRow: is the max row
   105  // commitTSVec: is the commit ts vector
   106  // abortVec: is the abort vector
   107  // aborts: is the aborted bitmap
   108  func (n *AppendMVCCHandle) CollectAppendLocked(
   109  	start, end types.TS, mp *mpool.MPool,
   110  ) (
   111  	minRow, maxRow uint32,
   112  	commitTSVec, abortVec containers.Vector,
   113  	aborts *nulls.Bitmap,
   114  ) {
   115  	startOffset, node := n.appends.GetNodeToReadByPrepareTS(start)
   116  	if node != nil {
   117  		prepareTS := node.GetPrepare()
   118  		if prepareTS.Less(&start) {
   119  			startOffset++
   120  		}
   121  	}
   122  	endOffset, node := n.appends.GetNodeToReadByPrepareTS(end)
   123  	if node == nil || startOffset > endOffset {
   124  		return
   125  	}
   126  	minRow = n.appends.GetNodeByOffset(startOffset).startRow
   127  	maxRow = node.maxRow
   128  
   129  	aborts = &nulls.Bitmap{}
   130  	commitTSVec = containers.MakeVector(types.T_TS.ToType(), mp)
   131  	abortVec = containers.MakeVector(types.T_bool.ToType(), mp)
   132  	n.appends.LoopOffsetRange(
   133  		startOffset,
   134  		endOffset,
   135  		func(node *AppendNode) bool {
   136  			txn := node.GetTxn()
   137  			if txn != nil {
   138  				n.RUnlock()
   139  				txn.GetTxnState(true)
   140  				n.RLock()
   141  			}
   142  			if node.IsAborted() {
   143  				aborts.AddRange(uint64(node.startRow), uint64(node.maxRow))
   144  			}
   145  			for i := 0; i < int(node.maxRow-node.startRow); i++ {
   146  				commitTSVec.Append(node.GetCommitTS(), false)
   147  				abortVec.Append(node.IsAborted(), false)
   148  			}
   149  			return true
   150  		})
   151  	return
   152  }
   153  
   154  // it is used to get the visible max row for a txn
   155  // maxrow: is the max row that the txn can see
   156  // visible: is true if the txn can see any row
   157  // holes: is the bitmap of the holes that the txn cannot see
   158  // holes exists only if any append node was rollbacked
   159  func (n *AppendMVCCHandle) GetVisibleRowLocked(
   160  	ctx context.Context,
   161  	txn txnif.TxnReader,
   162  ) (maxrow uint32, visible bool, holes *nulls.Bitmap, err error) {
   163  	var holesMax uint32
   164  	anToWait := make([]*AppendNode, 0)
   165  	txnToWait := make([]txnif.TxnReader, 0)
   166  	n.appends.ForEach(func(an *AppendNode) bool {
   167  		needWait, waitTxn := an.NeedWaitCommitting(txn.GetStartTS())
   168  		if needWait {
   169  			anToWait = append(anToWait, an)
   170  			txnToWait = append(txnToWait, waitTxn)
   171  			return true
   172  		}
   173  		if an.IsVisible(txn) {
   174  			visible = true
   175  			maxrow = an.maxRow
   176  		} else {
   177  			if holes == nil {
   178  				holes = nulls.NewWithSize(int(an.maxRow) + 1)
   179  			}
   180  			holes.AddRange(uint64(an.startRow), uint64(an.maxRow))
   181  			if holesMax < an.maxRow {
   182  				holesMax = an.maxRow
   183  			}
   184  		}
   185  		startTS := txn.GetStartTS()
   186  		return !an.Prepare.Greater(&startTS)
   187  	}, true)
   188  	if len(anToWait) != 0 {
   189  		n.RUnlock()
   190  		for _, txn := range txnToWait {
   191  			txn.GetTxnState(true)
   192  		}
   193  		n.RLock()
   194  	}
   195  	for _, an := range anToWait {
   196  		if an.IsVisible(txn) {
   197  			visible = true
   198  			if maxrow < an.maxRow {
   199  				maxrow = an.maxRow
   200  			}
   201  		} else {
   202  			if holes == nil {
   203  				holes = nulls.NewWithSize(int(an.maxRow) + 1)
   204  			}
   205  			holes.AddRange(uint64(an.startRow), uint64(an.maxRow))
   206  			if holesMax < an.maxRow {
   207  				holesMax = an.maxRow
   208  			}
   209  		}
   210  	}
   211  	if !holes.IsEmpty() {
   212  		for i := uint64(maxrow); i < uint64(holesMax); i++ {
   213  			holes.Del(i)
   214  		}
   215  	}
   216  	return
   217  }
   218  
   219  // it collects all append nodes that are prepared before the given ts
   220  // foreachFn is called for each append node that is prepared before the given ts
   221  func (n *AppendMVCCHandle) CollectUncommittedANodesPreparedBefore(
   222  	ts types.TS,
   223  	foreachFn func(*AppendNode),
   224  ) (anyWaitable bool) {
   225  	if n.appends.IsEmpty() {
   226  		return
   227  	}
   228  	n.appends.ForEach(func(an *AppendNode) bool {
   229  		needWait, txn := an.NeedWaitCommitting(ts)
   230  		if txn == nil {
   231  			return false
   232  		}
   233  		if needWait {
   234  			foreachFn(an)
   235  			anyWaitable = true
   236  		}
   237  		return true
   238  	}, false)
   239  	return
   240  }
   241  
   242  func (n *AppendMVCCHandle) OnReplayAppendNode(an *AppendNode) {
   243  	an.mvcc = n
   244  	n.appends.InsertNode(an)
   245  }
   246  
   247  // AddAppendNodeLocked add a new appendnode to the list.
   248  func (n *AppendMVCCHandle) AddAppendNodeLocked(
   249  	txn txnif.AsyncTxn,
   250  	startRow uint32,
   251  	maxRow uint32,
   252  ) (an *AppendNode, created bool) {
   253  	if n.appends.IsEmpty() || !n.appends.GetUpdateNodeLocked().IsSameTxn(txn) {
   254  		// if the appends is empty or the last appendnode is not of the same txn,
   255  		// create a new appendnode and append it to the list.
   256  		an = NewAppendNode(txn, startRow, maxRow, n)
   257  		n.appends.InsertNode(an)
   258  		created = true
   259  	} else {
   260  		// if the last appendnode is of the same txn, update the maxrow of the last appendnode.
   261  		an = n.appends.GetUpdateNodeLocked()
   262  		created = false
   263  		an.SetMaxRow(maxRow)
   264  	}
   265  	return
   266  }
   267  
   268  // Reschedule until all appendnode is committed.
   269  // Pending appendnode is not visible for compaction txn.
   270  func (n *AppendMVCCHandle) PrepareCompactLocked() bool {
   271  	return n.allAppendsCommittedLocked()
   272  }
   273  func (n *AppendMVCCHandle) PrepareCompact() bool {
   274  	n.RLock()
   275  	defer n.RUnlock()
   276  	return n.allAppendsCommittedLocked()
   277  }
   278  
   279  func (n *AppendMVCCHandle) GetLatestAppendPrepareTSLocked() types.TS {
   280  	return n.appends.GetUpdateNodeLocked().Prepare
   281  }
   282  
   283  // check if all appendnodes are committed.
   284  func (n *AppendMVCCHandle) allAppendsCommittedLocked() bool {
   285  	if n.appends == nil {
   286  		logutil.Warnf("[MetadataCheck] appends mvcc is nil, obj %v, has dropped %v, deleted at %v",
   287  			n.meta.ID.String(),
   288  			n.meta.HasDropCommittedLocked(),
   289  			n.meta.GetDeleteAtLocked().ToString())
   290  		return false
   291  	}
   292  	return n.appends.IsCommitted()
   293  }
   294  
   295  // DeleteAppendNodeLocked deletes the appendnode from the append list.
   296  // it is called when txn of the appendnode is aborted.
   297  func (n *AppendMVCCHandle) DeleteAppendNodeLocked(node *AppendNode) {
   298  	n.appends.DeleteNode(node)
   299  }
   300  
   301  func (n *AppendMVCCHandle) SetAppendListener(l func(txnif.AppendNode) error) {
   302  	n.appendListener = l
   303  }
   304  
   305  func (n *AppendMVCCHandle) GetAppendListener() func(txnif.AppendNode) error {
   306  	return n.appendListener
   307  }
   308  
   309  // AllAppendsCommittedBefore returns true if all appendnode is committed before ts.
   310  func (n *AppendMVCCHandle) AllAppendsCommittedBefore(ts types.TS) bool {
   311  	// get the latest appendnode
   312  	anode := n.appends.GetUpdateNodeLocked()
   313  	if anode == nil {
   314  		return false
   315  	}
   316  
   317  	// if the latest appendnode is not committed, return false
   318  	if !anode.IsCommitted() {
   319  		return false
   320  	}
   321  
   322  	// check if the latest appendnode is committed before ts
   323  	commitTS := anode.GetCommitTS()
   324  	return commitTS.Less(&ts)
   325  }
   326  
   327  func (n *AppendMVCCHandle) StringLocked() string {
   328  	return n.appends.StringLocked()
   329  }
   330  
   331  func (n *AppendMVCCHandle) EstimateMemSizeLocked() int {
   332  	asize := 0
   333  	if n.appends != nil {
   334  		asize += len(n.appends.MVCC) * AppendNodeApproxSize
   335  	}
   336  	return asize
   337  }
   338  
   339  // GetTotalRow is only for replay
   340  func (n *AppendMVCCHandle) GetTotalRow() uint32 {
   341  	an := n.appends.GetUpdateNodeLocked()
   342  	if an == nil {
   343  		return 0
   344  	}
   345  	return an.maxRow
   346  }
   347  
   348  func (n *AppendMVCCHandle) GetID() *common.ID {
   349  	return n.meta.AsCommonID()
   350  }
   351  
   352  type ObjectMVCCHandle struct {
   353  	*sync.RWMutex
   354  	deletes         map[uint16]*MVCCHandle
   355  	meta            *catalog.ObjectEntry
   356  	deletesListener func(uint64, types.TS) error
   357  }
   358  
   359  func NewObjectMVCCHandle(meta *catalog.ObjectEntry) *ObjectMVCCHandle {
   360  	node := &ObjectMVCCHandle{
   361  		RWMutex: meta.RWMutex,
   362  		meta:    meta,
   363  		deletes: make(map[uint16]*MVCCHandle),
   364  	}
   365  	node.UpgradeAllDeleteChain()
   366  	node.SetDeletesListener(node.OnApplyDelete)
   367  	return node
   368  }
   369  func (n *ObjectMVCCHandle) OnApplyDelete(
   370  	deleted uint64,
   371  	ts types.TS) (err error) {
   372  	n.meta.GetTable().RemoveRows(deleted)
   373  	return
   374  }
   375  func (n *ObjectMVCCHandle) GetOrCreateDeleteChainLocked(blkID uint16) *MVCCHandle {
   376  	deletes := n.deletes[blkID]
   377  	if deletes == nil {
   378  		deletes = NewMVCCHandle(n, blkID)
   379  		n.deletes[blkID] = deletes
   380  	}
   381  	return deletes
   382  }
   383  func (n *ObjectMVCCHandle) TryGetDeleteChain(blkID uint16) *MVCCHandle {
   384  	deletes := n.deletes[blkID]
   385  	return deletes
   386  }
   387  func (n *ObjectMVCCHandle) SetDeletesListener(l func(uint64, types.TS) error) {
   388  	n.deletesListener = l
   389  }
   390  
   391  func (n *ObjectMVCCHandle) GetDeletesListener() func(uint64, types.TS) error {
   392  	return n.deletesListener
   393  }
   394  
   395  func (n *ObjectMVCCHandle) GetChangeIntentionCntLocked() uint32 {
   396  	changes := uint32(0)
   397  	for _, deletes := range n.deletes {
   398  		changes += deletes.GetChangeIntentionCnt()
   399  	}
   400  	return changes
   401  }
   402  func (n *ObjectMVCCHandle) IsDeletedLocked(
   403  	row uint32, txn txnif.TxnReader, blkID uint16,
   404  ) (bool, error) {
   405  	deletes := n.TryGetDeleteChain(blkID)
   406  	if deletes == nil {
   407  		return false, nil
   408  	}
   409  	return deletes.IsDeletedLocked(row, txn)
   410  }
   411  
   412  func (n *ObjectMVCCHandle) UpgradeAllDeleteChain() {
   413  	for _, deletes := range n.deletes {
   414  		deletes.upgradeDeleteChain()
   415  	}
   416  }
   417  func (n *ObjectMVCCHandle) GetDeltaPersistedTS() types.TS {
   418  	persisted := types.TS{}
   419  	for _, deletes := range n.deletes {
   420  		ts := deletes.getDeltaPersistedTSLocked()
   421  		if ts.Greater(&persisted) {
   422  			persisted = ts
   423  		}
   424  	}
   425  	return persisted
   426  }
   427  
   428  func (n *ObjectMVCCHandle) UpgradeDeleteChain(blkID uint16) {
   429  	deletes := n.deletes[blkID]
   430  	if deletes == nil {
   431  		return
   432  	}
   433  	deletes.upgradeDeleteChain()
   434  }
   435  
   436  // for test
   437  func (n *ObjectMVCCHandle) UpgradeDeleteChainByTSLocked(ts types.TS) {
   438  	for _, deletes := range n.deletes {
   439  		deletes.upgradeDeleteChainByTSLocked(ts)
   440  	}
   441  }
   442  
   443  func (n *ObjectMVCCHandle) EstimateMemSizeLocked() (dsize int) {
   444  	for _, deletes := range n.deletes {
   445  		dsize += deletes.EstimateMemSizeLocked()
   446  	}
   447  	return
   448  }
   449  
   450  func (n *ObjectMVCCHandle) GetDeltaLocAndCommitTS(blkID uint16) (loc objectio.Location, start, end types.TS) {
   451  	n.RLock()
   452  	defer n.RUnlock()
   453  	deletes := n.deletes[blkID]
   454  	if deletes == nil {
   455  		return
   456  	}
   457  	return deletes.GetDeltaLocAndCommitTSLocked()
   458  }
   459  func (n *ObjectMVCCHandle) GetDeltaLocAndCommitTSByTxn(blkID uint16, txn txnif.TxnReader) (objectio.Location, types.TS) {
   460  	deletes := n.deletes[blkID]
   461  	if deletes == nil {
   462  		return nil, types.TS{}
   463  	}
   464  	return deletes.GetDeltaLocAndCommitTSByTxn(txn)
   465  }
   466  
   467  func (n *ObjectMVCCHandle) StringLocked(level common.PPLevel, depth int, prefix string) string {
   468  	s := ""
   469  	for _, deletes := range n.deletes {
   470  		s = fmt.Sprintf("%s%s", s, deletes.StringLocked(level, depth+1, prefix))
   471  	}
   472  	return s
   473  }
   474  
   475  func (n *ObjectMVCCHandle) StringBlkLocked(level common.PPLevel, depth int, prefix string, blkid int) string {
   476  	s := ""
   477  	if d, exist := n.deletes[uint16(blkid)]; exist {
   478  		s = fmt.Sprintf("%s%s", s, d.StringLocked(level, depth+1, prefix))
   479  	}
   480  	return s
   481  }
   482  
   483  func (n *ObjectMVCCHandle) GetDeleteCnt() uint32 {
   484  	cnt := uint32(0)
   485  	for _, deletes := range n.deletes {
   486  		cnt += deletes.GetDeleteCnt()
   487  	}
   488  	return cnt
   489  }
   490  func (n *ObjectMVCCHandle) HasDeleteIntentsPreparedIn(from, to types.TS) (found, isPersist bool) {
   491  	for _, deletes := range n.deletes {
   492  		found, isPersist = deletes.GetDeleteChain().HasDeleteIntentsPreparedInLocked(from, to)
   493  		if found {
   494  			return
   495  		}
   496  	}
   497  	return
   498  }
   499  func (n *ObjectMVCCHandle) HasInMemoryDeleteIntentsPreparedInByBlock(blkID uint16, from, to types.TS) (found, isPersist bool) {
   500  	mvcc := n.deletes[blkID]
   501  	if mvcc == nil {
   502  		return false, false
   503  	}
   504  	if mvcc.deletes.mask.IsEmpty() {
   505  		return false, false
   506  	}
   507  	found, isPersist = mvcc.GetDeleteChain().HasDeleteIntentsPreparedInLocked(from, to)
   508  	return
   509  }
   510  
   511  func (n *ObjectMVCCHandle) ReplayDeltaLoc(vMVCCNode any, blkID uint16) {
   512  	mvccNode := vMVCCNode.(*catalog.MVCCNode[*catalog.MetadataMVCCNode])
   513  	mvcc := n.GetOrCreateDeleteChainLocked(blkID)
   514  	mvcc.ReplayDeltaLoc(mvccNode)
   515  }
   516  func (n *ObjectMVCCHandle) InMemoryDeletesExisted() bool {
   517  	for _, deletes := range n.deletes {
   518  		if !deletes.deletes.mask.IsEmpty() {
   519  			return true
   520  		}
   521  	}
   522  	return false
   523  }
   524  func (n *ObjectMVCCHandle) GetObject() any {
   525  	return n.meta
   526  }
   527  func (n *ObjectMVCCHandle) GetLatestDeltaloc(blkOffset uint16) objectio.Location {
   528  	mvcc := n.TryGetDeleteChain(blkOffset)
   529  	if mvcc == nil {
   530  		return nil
   531  	}
   532  	return mvcc.deltaloc.GetLatestNodeLocked().BaseNode.DeltaLoc
   533  }
   534  func (n *ObjectMVCCHandle) GetLatestMVCCNode(blkOffset uint16) *catalog.MVCCNode[*catalog.MetadataMVCCNode] {
   535  	mvcc := n.TryGetDeleteChain(blkOffset)
   536  	if mvcc == nil {
   537  		return nil
   538  	}
   539  	return mvcc.deltaloc.GetLatestNodeLocked()
   540  }
   541  func (n *ObjectMVCCHandle) VisitDeletes(
   542  	ctx context.Context,
   543  	start, end types.TS,
   544  	deltalocBat *containers.Batch,
   545  	tnInsertBat *containers.Batch,
   546  	skipInMemory bool) (delBatch *containers.Batch, deltalocStart, deltalocEnd int, err error) {
   547  	n.RLock()
   548  	defer n.RUnlock()
   549  	deltalocStart = deltalocBat.Length()
   550  	for blkOffset, mvcc := range n.deletes {
   551  		newStart := start
   552  		nodes := mvcc.deltaloc.ClonePreparedInRangeLocked(start, end)
   553  		var skipData bool
   554  		if len(nodes) != 0 {
   555  			blkID := objectio.NewBlockidWithObjectID(&n.meta.ID, blkOffset)
   556  			for _, node := range nodes {
   557  				VisitDeltaloc(deltalocBat, tnInsertBat, n.meta, blkID, node, node.End, node.CreatedAt)
   558  			}
   559  			newest := nodes[len(nodes)-1]
   560  			// block has newer delta data on s3, no need to collect data
   561  			startTS := newest.GetStart()
   562  			skipData = startTS.GreaterEq(&end)
   563  			newStart = newest.GetStart()
   564  		}
   565  		if !skipData && !skipInMemory {
   566  			deletes := n.deletes[blkOffset]
   567  			n.RUnlock()
   568  			delBat, err := deletes.CollectDeleteInRangeAfterDeltalocation(ctx, newStart, end, false, common.LogtailAllocator)
   569  			n.RLock()
   570  			if err != nil {
   571  				if delBatch != nil {
   572  					delBatch.Close()
   573  				}
   574  				delBat.Close()
   575  				return nil, 0, 0, err
   576  			}
   577  			if delBat != nil && delBat.Length() > 0 {
   578  				if delBatch == nil {
   579  					delBatch = containers.NewBatch()
   580  					delBatch.AddVector(
   581  						catalog.AttrRowID,
   582  						containers.MakeVector(types.T_Rowid.ToType(), common.LogtailAllocator),
   583  					)
   584  					delBatch.AddVector(
   585  						catalog.AttrCommitTs,
   586  						containers.MakeVector(types.T_TS.ToType(), common.LogtailAllocator),
   587  					)
   588  					delBatch.AddVector(
   589  						catalog.AttrPKVal,
   590  						containers.MakeVector(*delBat.GetVectorByName(catalog.AttrPKVal).GetType(), common.LogtailAllocator),
   591  					)
   592  				}
   593  				delBatch.Extend(delBat)
   594  				// delBatch is freed, don't use anymore
   595  				delBat.Close()
   596  			}
   597  		}
   598  	}
   599  	deltalocEnd = deltalocBat.Length()
   600  	return
   601  }
   602  
   603  func VisitDeltaloc(bat, tnBatch *containers.Batch, object *catalog.ObjectEntry, blkID *objectio.Blockid, node *catalog.MVCCNode[*catalog.MetadataMVCCNode], commitTS, createTS types.TS) {
   604  	is_sorted := false
   605  	if !object.IsAppendable() && object.GetSchema().HasSortKey() {
   606  		is_sorted = true
   607  	}
   608  	bat.GetVectorByName(pkgcatalog.BlockMeta_ID).Append(*blkID, false)
   609  	bat.GetVectorByName(pkgcatalog.BlockMeta_EntryState).Append(object.IsAppendable(), false)
   610  	bat.GetVectorByName(pkgcatalog.BlockMeta_Sorted).Append(is_sorted, false)
   611  	bat.GetVectorByName(pkgcatalog.BlockMeta_MetaLoc).Append([]byte(node.BaseNode.MetaLoc), false)
   612  	bat.GetVectorByName(pkgcatalog.BlockMeta_DeltaLoc).Append([]byte(node.BaseNode.DeltaLoc), false)
   613  	bat.GetVectorByName(pkgcatalog.BlockMeta_CommitTs).Append(commitTS, false)
   614  	bat.GetVectorByName(pkgcatalog.BlockMeta_SegmentID).Append(*object.ID.Segment(), false)
   615  	bat.GetVectorByName(pkgcatalog.BlockMeta_MemTruncPoint).Append(node.Start, false)
   616  	bat.GetVectorByName(catalog.AttrCommitTs).Append(createTS, false)
   617  	bat.GetVectorByName(catalog.AttrRowID).Append(objectio.HackBlockid2Rowid(blkID), false)
   618  
   619  	// When pull and push, it doesn't collect tn batch
   620  	if tnBatch != nil {
   621  		tnBatch.GetVectorByName(catalog.SnapshotAttr_DBID).Append(object.GetTable().GetDB().ID, false)
   622  		tnBatch.GetVectorByName(catalog.SnapshotAttr_TID).Append(object.GetTable().ID, false)
   623  		node.TxnMVCCNode.AppendTuple(tnBatch)
   624  	}
   625  }
   626  
   627  type DeltalocChain struct {
   628  	mvcc *MVCCHandle
   629  	*catalog.BaseEntryImpl[*catalog.MetadataMVCCNode]
   630  }
   631  
   632  func NewDeltalocChain(mvcc *MVCCHandle) *DeltalocChain {
   633  	delChain := &DeltalocChain{
   634  		mvcc:          mvcc,
   635  		BaseEntryImpl: catalog.NewBaseEntry(func() *catalog.MetadataMVCCNode { return &catalog.MetadataMVCCNode{} }),
   636  	}
   637  	delChain.RWMutex = mvcc.RWMutex
   638  	return delChain
   639  }
   640  func (d *DeltalocChain) PrepareCommit() (err error) {
   641  	d.Lock()
   642  	defer d.Unlock()
   643  	node := d.GetLatestNodeLocked()
   644  	if node.BaseNode.NeedCheckDeleteChainWhenCommit {
   645  		if found, _ := d.mvcc.GetDeleteChain().HasDeleteIntentsPreparedInLocked(node.Start, node.Txn.GetPrepareTS()); found {
   646  			return txnif.ErrTxnNeedRetry
   647  		}
   648  	}
   649  	_, err = node.TxnMVCCNode.PrepareCommit()
   650  	if err != nil {
   651  		return
   652  	}
   653  	return
   654  }
   655  func (d *DeltalocChain) Is1PC() bool { return false }
   656  func (d *DeltalocChain) MakeCommand(id uint32) (cmd txnif.TxnCmd, err error) {
   657  	return catalog.NewDeltalocCmd(id, catalog.IOET_WALTxnCommand_Block, d.mvcc.GetID(), d.BaseEntryImpl), nil
   658  }
   659  func (d *DeltalocChain) PrepareRollback() error {
   660  	d.RLock()
   661  	node := d.GetLatestNodeLocked()
   662  	d.RUnlock()
   663  	// If it's deleted by deltaloc, reset persisted mask when rollback
   664  	if node.BaseNode.NeedCheckDeleteChainWhenCommit {
   665  		d.Lock()
   666  		d.mvcc.GetDeleteChain().ResetPersistedMask()
   667  		d.Unlock()
   668  	}
   669  	_, err := d.BaseEntryImpl.PrepareRollback()
   670  	return err
   671  }
   672  func (d *DeltalocChain) Set1PC() {}
   673  
   674  func (d *DeltalocChain) GetBlockID() *objectio.Blockid {
   675  	return objectio.NewBlockidWithObjectID(&d.mvcc.meta.ID, d.mvcc.blkID)
   676  }
   677  func (d *DeltalocChain) GetMeta() *catalog.ObjectEntry { return d.mvcc.meta }
   678  
   679  type MVCCHandle struct {
   680  	*ObjectMVCCHandle
   681  	changes     atomic.Uint32
   682  	deletes     *DeleteChain
   683  	deltaloc    *DeltalocChain
   684  	blkID       uint16
   685  	persistedTS types.TS
   686  }
   687  
   688  func NewMVCCHandle(meta *ObjectMVCCHandle, blkID uint16) *MVCCHandle {
   689  	node := &MVCCHandle{
   690  		ObjectMVCCHandle: meta,
   691  		blkID:            blkID,
   692  	}
   693  	if meta == nil {
   694  		return node
   695  	}
   696  	node.deletes = NewDeleteChain(node.RWMutex, node)
   697  	node.deltaloc = NewDeltalocChain(node)
   698  	return node
   699  }
   700  
   701  // ==========================================================
   702  // *************** All common related APIs *****************
   703  // ==========================================================
   704  
   705  func (n *MVCCHandle) GetID() *common.ID {
   706  	id := n.meta.AsCommonID()
   707  	id.SetBlockOffset(n.blkID)
   708  	return id
   709  }
   710  func (n *MVCCHandle) GetEntry() *catalog.ObjectEntry { return n.meta }
   711  
   712  func (n *MVCCHandle) StringLocked(level common.PPLevel, depth int, prefix string) string {
   713  	inMemoryCount := 0
   714  	if n.deletes.DepthLocked() > 0 {
   715  		// s = fmt.Sprintf("%s%s", s, n.deletes.StringLocked())
   716  		inMemoryCount = n.deletes.mask.GetCardinality()
   717  	}
   718  	s := fmt.Sprintf("%sBLK[%d]InMem:%d\n", common.RepeatStr("\t", depth), n.blkID, inMemoryCount)
   719  	if level > common.PPL3 {
   720  		if imemChain := n.deletes.StringLocked(); imemChain != "" {
   721  			s = fmt.Sprintf("%s%s", s, imemChain)
   722  		}
   723  	}
   724  	if n.deltaloc.Depth() > 0 {
   725  		s = fmt.Sprintf("%s%s", s, n.deltaloc.StringLocked())
   726  	}
   727  	s = s + "\n"
   728  	return s
   729  }
   730  
   731  func (n *MVCCHandle) EstimateMemSizeLocked() (dsize int) {
   732  	dsize = n.deletes.EstimateMemSizeLocked()
   733  	return dsize + MVCCHandleApproxSize
   734  }
   735  
   736  // ==========================================================
   737  // *************** All deletes related APIs *****************
   738  // ==========================================================
   739  
   740  func (n *MVCCHandle) getDeltaPersistedTSLocked() types.TS {
   741  	persisted := types.TS{}
   742  	n.deltaloc.LoopChainLocked(func(m *catalog.MVCCNode[*catalog.MetadataMVCCNode]) bool {
   743  		if !m.BaseNode.DeltaLoc.IsEmpty() && m.IsCommitted() {
   744  			persisted = m.GetStart()
   745  			return false
   746  		}
   747  		return true
   748  	})
   749  	return persisted
   750  }
   751  
   752  func (n *MVCCHandle) upgradeDeleteChainByTSLocked(flushed types.TS) {
   753  	if n.persistedTS.Equal(&flushed) {
   754  		return
   755  	}
   756  	n.deletes = n.deletes.shrinkDeleteChainByTSLocked(flushed)
   757  
   758  	n.persistedTS = flushed
   759  }
   760  
   761  func (n *MVCCHandle) upgradeDeleteChain() {
   762  	persisted := n.getDeltaPersistedTSLocked()
   763  	n.upgradeDeleteChainByTSLocked(persisted)
   764  }
   765  
   766  func (n *MVCCHandle) IncChangeIntentionCnt() {
   767  	n.changes.Add(1)
   768  }
   769  
   770  func (n *MVCCHandle) DecChangeIntentionCnt() {
   771  	n.changes.Add(^uint32(0))
   772  }
   773  
   774  // GetChangeIntentionCnt returns the number of operation of delete, which is updated before commiting.
   775  // Note: Now it is ** only ** used in checkpointe runner to check whether this block has any chance to be flushed
   776  func (n *MVCCHandle) GetChangeIntentionCnt() uint32 {
   777  	return n.changes.Load()
   778  }
   779  
   780  // GetDeleteCnt returns committed deleted rows
   781  func (n *MVCCHandle) GetDeleteCnt() uint32 {
   782  	return n.deletes.GetDeleteCnt()
   783  }
   784  
   785  // it checks whether there is any delete in the range [start, end)
   786  // ts is not used for now
   787  func (n *MVCCHandle) CheckNotDeleted(start, end uint32, ts types.TS) error {
   788  	return n.deletes.PrepareRangeDelete(start, end, ts)
   789  }
   790  
   791  func (n *MVCCHandle) CreateDeleteNode(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode {
   792  	return n.deletes.AddNodeLocked(txn, deleteType)
   793  }
   794  
   795  func (n *MVCCHandle) OnReplayDeleteNode(deleteNode txnif.DeleteNode) {
   796  	n.deletes.OnReplayNode(deleteNode.(*DeleteNode))
   797  }
   798  
   799  func (n *MVCCHandle) GetDeleteChain() *DeleteChain {
   800  	return n.deletes
   801  }
   802  
   803  func (n *MVCCHandle) IsDeletedLocked(
   804  	row uint32, txn txnif.TxnReader,
   805  ) (bool, error) {
   806  	return n.deletes.IsDeleted(row, txn, n.RWMutex)
   807  }
   808  
   809  // it collects all deletes in the range [start, end)
   810  func (n *MVCCHandle) CollectDeleteLocked(
   811  	start, end types.TS, pkType types.Type, mp *mpool.MPool,
   812  ) (rowIDVec, commitTSVec, pkVec, abortVec containers.Vector,
   813  	aborts *nulls.Bitmap, deletes []uint32, minTS, persistedTS types.TS,
   814  ) {
   815  	persistedTS = n.persistedTS
   816  	if n.deletes.IsEmpty() {
   817  		return
   818  	}
   819  	if !n.ExistDeleteInRangeLocked(start, end) {
   820  		return
   821  	}
   822  
   823  	for {
   824  		needWaitFound := false
   825  		if rowIDVec != nil {
   826  			rowIDVec.Close()
   827  		}
   828  		rowIDVec = containers.MakeVector(types.T_Rowid.ToType(), mp)
   829  		if commitTSVec != nil {
   830  			commitTSVec.Close()
   831  		}
   832  		commitTSVec = containers.MakeVector(types.T_TS.ToType(), mp)
   833  		if pkVec != nil {
   834  			pkVec.Close()
   835  		}
   836  		pkVec = containers.MakeVector(pkType, mp)
   837  		aborts = &nulls.Bitmap{}
   838  		id := objectio.NewBlockidWithObjectID(&n.meta.ID, n.blkID)
   839  		n.deletes.LoopChainLocked(
   840  			func(node *DeleteNode) bool {
   841  				needWait, txn := node.NeedWaitCommitting(end.Next())
   842  				if needWait {
   843  					n.RUnlock()
   844  					txn.GetTxnState(true)
   845  					n.RLock()
   846  					needWaitFound = true
   847  					return false
   848  				}
   849  				if node.nt == NT_Persisted {
   850  					return true
   851  				}
   852  				in, before := node.PreparedIn(start, end)
   853  				if in {
   854  					it := node.mask.Iterator()
   855  					if node.IsAborted() {
   856  						it := node.mask.Iterator()
   857  						for it.HasNext() {
   858  							row := it.Next()
   859  							nulls.Add(aborts, uint64(row))
   860  						}
   861  					}
   862  					for it.HasNext() {
   863  						row := it.Next()
   864  						rowIDVec.Append(*objectio.NewRowid(id, row), false)
   865  						commitTSVec.Append(node.GetEnd(), false)
   866  						// for deleteNode V1,rowid2PK is nil after restart
   867  						if node.version < IOET_WALTxnCommand_DeleteNode_V2 {
   868  							if deletes == nil {
   869  								deletes = make([]uint32, 0)
   870  							}
   871  							deletes = append(deletes, row)
   872  						} else {
   873  							pkVec.Append(node.rowid2PK[row].Get(0), false)
   874  						}
   875  						if minTS.IsEmpty() {
   876  							minTS = node.GetEnd()
   877  						} else {
   878  							end := node.GetEnd()
   879  							if minTS.Greater(&end) {
   880  								minTS = node.GetEnd()
   881  							}
   882  						}
   883  					}
   884  				}
   885  				return !before
   886  			})
   887  		if !needWaitFound {
   888  			break
   889  		}
   890  	}
   891  	abortVec = containers.NewConstFixed[bool](types.T_bool.ToType(), false, rowIDVec.Length(), containers.Options{Allocator: mp})
   892  	return
   893  }
   894  
   895  func (n *MVCCHandle) InMemoryCollectDeleteInRange(
   896  	ctx context.Context,
   897  	start, end types.TS,
   898  	withAborted bool,
   899  	mp *mpool.MPool,
   900  ) (bat *containers.Batch, minTS, persisitedTS types.TS, err error) {
   901  	n.RLock()
   902  	schema := n.meta.GetSchemaLocked()
   903  	pkDef := schema.GetPrimaryKey()
   904  	rowID, ts, pk, abort, abortedMap, deletes, minTS, persisitedTS := n.CollectDeleteLocked(start, end, pkDef.Type, mp)
   905  	n.RUnlock()
   906  	if rowID == nil {
   907  		return
   908  	}
   909  	// for deleteNode version less than 2, pk doesn't exist in memory
   910  	// collect pk by block.Foreach
   911  	if len(deletes) != 0 {
   912  		logutil.Infof("visit deletes: collect pk by load, obj is %v", n.meta.ID.String())
   913  		pkIdx := pkDef.Idx
   914  		data := n.meta.GetObjectData()
   915  		data.Foreach(ctx, schema, n.blkID, pkIdx, func(v any, isNull bool, row int) error {
   916  			pk.Append(v, false)
   917  			return nil
   918  		}, deletes, mp)
   919  	}
   920  	// batch: rowID, ts, pkVec, abort
   921  	bat = containers.NewBatch()
   922  	bat.AddVector(catalog.PhyAddrColumnName, rowID)
   923  	bat.AddVector(catalog.AttrCommitTs, ts)
   924  	bat.AddVector(catalog.AttrPKVal, pk)
   925  	if withAborted {
   926  		bat.AddVector(catalog.AttrAborted, abort)
   927  	} else {
   928  		abort.Close()
   929  		bat.Deletes = abortedMap
   930  		bat.Compact()
   931  	}
   932  	return
   933  }
   934  
   935  // CollectDeleteInRangeAfterDeltalocation collects deletes after
   936  // a certain delta location and committed in [start,end]
   937  // When subscribe a table, it collects delta location, then it collects deletes.
   938  // To avoid collecting duplicate deletes,
   939  // it collects after start ts of the delta location.
   940  // If the delta location is from CN, deletes is committed after startTS.
   941  // CollectDeleteInRange still collect duplicate deletes.
   942  func (n *MVCCHandle) CollectDeleteInRangeAfterDeltalocation(
   943  	ctx context.Context,
   944  	start, end types.TS, // start is startTS of deltalocation
   945  	withAborted bool,
   946  	mp *mpool.MPool,
   947  ) (bat *containers.Batch, err error) {
   948  	// persisted is persistedTS of deletes of the blk
   949  	// it equals startTS of the last delta location
   950  	deletes, _, persisted, err := n.InMemoryCollectDeleteInRange(
   951  		ctx,
   952  		start,
   953  		end,
   954  		withAborted,
   955  		mp,
   956  	)
   957  	if err != nil {
   958  		return nil, err
   959  	}
   960  	// if persisted > start,
   961  	// there's another delta location committed.
   962  	// It includes more deletes than former delta location.
   963  	if persisted.Greater(&start) {
   964  		deletes, err = n.meta.GetObjectData().PersistedCollectDeleteInRange(
   965  			ctx,
   966  			deletes,
   967  			n.blkID,
   968  			start,
   969  			end,
   970  			withAborted,
   971  			mp,
   972  		)
   973  	}
   974  	if deletes != nil && deletes.Length() != 0 {
   975  		if bat == nil {
   976  			bat = containers.NewBatch()
   977  			bat.AddVector(catalog.AttrRowID, containers.MakeVector(types.T_Rowid.ToType(), mp))
   978  			bat.AddVector(catalog.AttrCommitTs, containers.MakeVector(types.T_TS.ToType(), mp))
   979  			bat.AddVector(catalog.AttrPKVal, containers.MakeVector(*deletes.GetVectorByName(catalog.AttrPKVal).GetType(), mp))
   980  			if withAborted {
   981  				bat.AddVector(catalog.AttrAborted, containers.MakeVector(types.T_bool.ToType(), mp))
   982  			}
   983  		}
   984  		bat.Extend(deletes)
   985  		deletes.Close()
   986  	}
   987  	return
   988  }
   989  
   990  // ExistDeleteInRange check if there is any delete in the range [start, end]
   991  // it loops the delete chain and check if there is any delete node in the range
   992  func (n *MVCCHandle) ExistDeleteInRangeLocked(start, end types.TS) (exist bool) {
   993  	for {
   994  		needWaitFound := false
   995  		n.deletes.LoopChainLocked(
   996  			func(node *DeleteNode) bool {
   997  				needWait, txn := node.NeedWaitCommitting(end.Next())
   998  				if needWait {
   999  					n.RUnlock()
  1000  					txn.GetTxnState(true)
  1001  					n.RLock()
  1002  					needWaitFound = true
  1003  					return false
  1004  				}
  1005  				in, before := node.PreparedIn(start, end)
  1006  				if in {
  1007  					exist = true
  1008  					return false
  1009  				}
  1010  				return !before
  1011  			})
  1012  		if !needWaitFound {
  1013  			break
  1014  		}
  1015  	}
  1016  
  1017  	return
  1018  }
  1019  
  1020  func (n *MVCCHandle) GetDeleteNodeByRow(row uint32) (an *DeleteNode) {
  1021  	return n.deletes.GetDeleteNodeByRow(row)
  1022  }
  1023  func (n *MVCCHandle) GetDeltaLocAndCommitTS() (objectio.Location, types.TS, types.TS) {
  1024  	n.RLock()
  1025  	defer n.RUnlock()
  1026  	return n.GetDeltaLocAndCommitTSLocked()
  1027  }
  1028  func (n *MVCCHandle) GetDeltaLocAndCommitTSLocked() (objectio.Location, types.TS, types.TS) {
  1029  	node := n.deltaloc.GetLatestNodeLocked()
  1030  	if node == nil {
  1031  		return nil, types.TS{}, types.TS{}
  1032  	}
  1033  	str := node.BaseNode.DeltaLoc
  1034  	committs := node.End
  1035  	startts := node.Start
  1036  	return str, startts, committs
  1037  }
  1038  func (n *MVCCHandle) GetDeltaLocAndCommitTSByTxn(txn txnif.TxnReader) (objectio.Location, types.TS) {
  1039  	n.RLock()
  1040  	defer n.RUnlock()
  1041  	node := n.deltaloc.GetVisibleNodeLocked(txn)
  1042  	if node == nil {
  1043  		return nil, types.TS{}
  1044  	}
  1045  	str := node.BaseNode.DeltaLoc
  1046  	ts := node.End
  1047  	return str, ts
  1048  }
  1049  func (n *MVCCHandle) isEmptyLocked() bool {
  1050  	if n.deltaloc.Depth() != 0 {
  1051  		return false
  1052  	}
  1053  	if !n.deletes.IsEmpty() {
  1054  		return false
  1055  	}
  1056  	return true
  1057  }
  1058  func (n *MVCCHandle) TryDeleteByDeltalocLocked(txn txnif.AsyncTxn, deltaLoc objectio.Location, needCheckWhenCommit bool) (entry txnif.TxnEntry, ok bool, err error) {
  1059  	if !n.isEmptyLocked() {
  1060  		return
  1061  	}
  1062  	_, entry, err = n.UpdateDeltaLocLocked(txn, deltaLoc, needCheckWhenCommit)
  1063  	if err != nil {
  1064  		return
  1065  	}
  1066  	bat, release, err := blockio.LoadTombstoneColumns(
  1067  		txn.GetContext(),
  1068  		[]uint16{0},
  1069  		nil,
  1070  		n.meta.GetObjectData().GetFs().Service,
  1071  		deltaLoc,
  1072  		nil,
  1073  	)
  1074  	defer release()
  1075  	if err == nil {
  1076  		ok = true
  1077  	}
  1078  	rowids := containers.ToTNVector(bat.Vecs[0], common.MutMemAllocator)
  1079  	defer rowids.Close()
  1080  	err = containers.ForeachVector(rowids, func(rowid types.Rowid, _ bool, row int) error {
  1081  		offset := rowid.GetRowOffset()
  1082  		n.deletes.persistedMask.Add(uint64(offset))
  1083  		return nil
  1084  	}, nil)
  1085  	if err == nil {
  1086  		ok = true
  1087  	}
  1088  	return
  1089  }
  1090  func (n *MVCCHandle) UpdateDeltaLocLocked(txn txnif.TxnReader, deltaloc objectio.Location, needCheckWhenCommit bool) (isNewNode bool, entry txnif.TxnEntry, err error) {
  1091  	needWait, txnToWait := n.deltaloc.NeedWaitCommittingLocked(txn.GetStartTS())
  1092  	if needWait {
  1093  		n.Unlock()
  1094  		txnToWait.GetTxnState(true)
  1095  		n.Lock()
  1096  	}
  1097  	err = n.deltaloc.CheckConflictLocked(txn)
  1098  	if err != nil {
  1099  		return
  1100  	}
  1101  	baseNode := &catalog.MetadataMVCCNode{
  1102  		DeltaLoc:                       deltaloc,
  1103  		NeedCheckDeleteChainWhenCommit: needCheckWhenCommit,
  1104  	}
  1105  	entry = n.deltaloc
  1106  
  1107  	if !n.deltaloc.IsEmptyLocked() {
  1108  		node := n.deltaloc.GetLatestNodeLocked()
  1109  		if node.IsSameTxn(txn) {
  1110  			node.BaseNode.Update(baseNode)
  1111  			return
  1112  		}
  1113  
  1114  	}
  1115  
  1116  	node := &catalog.MVCCNode[*catalog.MetadataMVCCNode]{
  1117  		EntryMVCCNode: &catalog.EntryMVCCNode{},
  1118  		BaseNode:      baseNode,
  1119  	}
  1120  	node.TxnMVCCNode = txnbase.NewTxnMVCCNodeWithTxn(txn)
  1121  	n.deltaloc.Insert(node)
  1122  	isNewNode = true
  1123  	return
  1124  }
  1125  
  1126  func (n *MVCCHandle) ReplayDeltaLoc(mvcc *catalog.MVCCNode[*catalog.MetadataMVCCNode]) {
  1127  	n.deltaloc.Insert(mvcc)
  1128  }