github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/tables/updates/delchain.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  	"fmt"
    19  	"sync"
    20  	"sync/atomic"
    21  
    22  	"github.com/matrixorigin/matrixone/pkg/container/nulls"
    23  	"github.com/matrixorigin/matrixone/pkg/container/types"
    24  	"github.com/matrixorigin/matrixone/pkg/logutil"
    25  	"github.com/matrixorigin/matrixone/pkg/objectio"
    26  
    27  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common"
    28  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/handle"
    29  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif"
    30  	"github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase"
    31  )
    32  
    33  func mockTxn() *txnbase.Txn {
    34  	txn := new(txnbase.Txn)
    35  	txn.TxnCtx = txnbase.NewTxnCtx(common.NewTxnIDAllocator().Alloc(), types.NextGlobalTsForTest(), types.TS{})
    36  	return txn
    37  }
    38  
    39  func MockTxnWithStartTS(ts types.TS) *txnbase.Txn {
    40  	txn := mockTxn()
    41  	txn.StartTS = ts
    42  	return txn
    43  }
    44  
    45  type DeleteChain struct {
    46  	*txnbase.MVCCChain[*DeleteNode]
    47  	mvcc          *MVCCHandle
    48  	links         map[uint32]*DeleteNode
    49  	cnt           atomic.Uint32
    50  	mask          *nulls.Bitmap // in memory del mask
    51  	persistedMask *nulls.Bitmap // persisted del mask
    52  }
    53  
    54  func NewDeleteChain(rwlocker *sync.RWMutex, mvcc *MVCCHandle) *DeleteChain {
    55  	if rwlocker == nil {
    56  		rwlocker = new(sync.RWMutex)
    57  	}
    58  	chain := &DeleteChain{
    59  		MVCCChain:     txnbase.NewMVCCChain((*DeleteNode).Less, NewEmptyDeleteNode, rwlocker),
    60  		links:         make(map[uint32]*DeleteNode),
    61  		mvcc:          mvcc,
    62  		mask:          &nulls.Bitmap{},
    63  		persistedMask: &nulls.Bitmap{},
    64  	}
    65  	return chain
    66  }
    67  func (chain *DeleteChain) AddDeleteCnt(cnt uint32) {
    68  	chain.cnt.Add(cnt)
    69  }
    70  
    71  // IsEmtpy indicates whether memory deletes are empty
    72  func (chain *DeleteChain) IsEmpty() bool {
    73  	return chain.mask.IsEmpty()
    74  }
    75  func (chain *DeleteChain) GetDeleteCnt() uint32 {
    76  	return chain.cnt.Load()
    77  }
    78  
    79  func (chain *DeleteChain) StringLocked() string {
    80  	msg := fmt.Sprintf("DeleteChain:%v", chain.mvcc.persistedTS.ToString())
    81  	line := 1
    82  	chain.LoopChainLocked(func(n *DeleteNode) bool {
    83  		msg = fmt.Sprintf("%s\n%d. %s", msg, line, n.StringLocked())
    84  		line++
    85  		return true
    86  	})
    87  	if line == 1 {
    88  		return ""
    89  	}
    90  	return msg
    91  }
    92  
    93  func (chain *DeleteChain) EstimateMemSizeLocked() int {
    94  	size := 0
    95  	if chain.mask != nil {
    96  		size += chain.mask.GetBitmap().Size()
    97  	}
    98  	if chain.persistedMask != nil {
    99  		size += chain.persistedMask.GetBitmap().Size()
   100  	}
   101  	size += int(float32(len(chain.links)*(4+8)) * 1.1) /*map size*/
   102  	size += chain.MVCCChain.Depth() * (DeleteNodeApproxSize + 16 /* link node overhead */)
   103  
   104  	return size + DeleteChainApproxSize
   105  }
   106  
   107  func (chain *DeleteChain) GetController() *MVCCHandle { return chain.mvcc }
   108  
   109  func (chain *DeleteChain) IsDeleted(row uint32, txn txnif.TxnReader, rwlocker *sync.RWMutex) (deleted bool, err error) {
   110  	deleteNode := chain.GetDeleteNodeByRow(row)
   111  	if deleteNode == nil {
   112  		return false, nil
   113  	}
   114  	needWait, waitTxn := deleteNode.NeedWaitCommitting(txn.GetStartTS())
   115  	if needWait {
   116  		rwlocker.RUnlock()
   117  		waitTxn.GetTxnState(true)
   118  		rwlocker.RLock()
   119  	}
   120  	return deleteNode.IsVisible(txn), nil
   121  }
   122  
   123  func (chain *DeleteChain) PrepareRangeDelete(start, end uint32, ts types.TS) (err error) {
   124  	if chain.hasOverLap(uint64(start), uint64(end)) {
   125  		err = txnif.ErrTxnWWConflict
   126  	}
   127  	return
   128  }
   129  
   130  func (chain *DeleteChain) hasOverLap(start, end uint64) bool {
   131  	if (chain.mask == nil || chain.mask.IsEmpty()) &&
   132  		(chain.persistedMask == nil || chain.persistedMask.IsEmpty()) {
   133  		return false
   134  	}
   135  	var yes bool
   136  	for i := start; i < end+1; i++ {
   137  		if chain.mask.Contains(i) || chain.persistedMask.Contains(i) {
   138  			yes = true
   139  			break
   140  		}
   141  	}
   142  	return yes
   143  }
   144  
   145  func (chain *DeleteChain) UpdateLocked(node *DeleteNode) {
   146  	chain.MVCC.Update(node.GenericDLNode)
   147  }
   148  
   149  func (chain *DeleteChain) RemoveNodeLocked(node txnif.DeleteNode) {
   150  	chain.MVCC.Delete(node.(*DeleteNode).GenericDLNode)
   151  	chain.deleteInMaskByNode(node)
   152  }
   153  
   154  func (chain *DeleteChain) deleteInMaskByNode(node txnif.DeleteNode) {
   155  	it := node.GetRowMaskRefLocked().Iterator()
   156  	for it.HasNext() {
   157  		row := it.Next()
   158  		chain.mask.Del(uint64(row))
   159  	}
   160  }
   161  
   162  func (chain *DeleteChain) insertInMaskByNode(node txnif.DeleteNode) {
   163  	it := node.GetRowMaskRefLocked().Iterator()
   164  	for it.HasNext() {
   165  		row := it.Next()
   166  		chain.mask.Add(uint64(row))
   167  	}
   168  }
   169  
   170  func (chain *DeleteChain) insertInMaskByRange(start, end uint32) {
   171  	chain.mask.AddRange(uint64(start), uint64(end+1))
   172  }
   173  
   174  func (chain *DeleteChain) DepthLocked() int { return chain.MVCC.Depth() }
   175  
   176  func (chain *DeleteChain) AddNodeLocked(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode {
   177  	node := NewDeleteNode(txn, deleteType, IOET_WALTxnCommand_DeleteNode_V2)
   178  	node.AttachTo(chain)
   179  	return node
   180  }
   181  
   182  func (chain *DeleteChain) AddPersistedNodeLocked(txn txnif.AsyncTxn, deltaloc objectio.Location) txnif.DeleteNode {
   183  	node := NewPersistedDeleteNode(txn, deltaloc)
   184  	node.AttachTo(chain)
   185  	node.setPersistedRows()
   186  	mask := node.chain.Load().mask
   187  	it := node.mask.Iterator()
   188  	for it.HasNext() {
   189  		row := it.Next()
   190  		mask.Add(uint64(row))
   191  	}
   192  	return node
   193  }
   194  
   195  func (chain *DeleteChain) InsertInDeleteView(row uint32, deleteNode *DeleteNode) {
   196  	if chain.links[row] != nil {
   197  		panic(fmt.Sprintf("row %d already in delete view", row))
   198  	}
   199  	chain.links[row] = deleteNode
   200  }
   201  func (chain *DeleteChain) DeleteInDeleteView(deleteNode *DeleteNode) {
   202  	it := deleteNode.mask.Iterator()
   203  	for it.HasNext() {
   204  		row := it.Next()
   205  		if chain.links[row] != deleteNode {
   206  			panic(fmt.Sprintf("row %d not in delete view", row))
   207  		}
   208  		delete(chain.links, row)
   209  	}
   210  }
   211  
   212  func (chain *DeleteChain) shrinkDeleteChainByTSLocked(flushed types.TS) *DeleteChain {
   213  	new := NewDeleteChain(chain.RWMutex, chain.mvcc)
   214  	new.persistedMask = chain.persistedMask
   215  
   216  	chain.LoopChainLocked(func(n *DeleteNode) bool {
   217  		if !n.IsVisibleByTS(flushed) {
   218  			if n.nt == NT_Persisted {
   219  				return false
   220  			}
   221  			n.AttachTo(new)
   222  			it := n.mask.Iterator()
   223  			for it.HasNext() {
   224  				row := it.Next()
   225  				new.InsertInDeleteView(row, n)
   226  				new.mask.Add(uint64(row))
   227  			}
   228  		} else {
   229  			it := n.mask.Iterator()
   230  			for it.HasNext() {
   231  				row := it.Next()
   232  				new.persistedMask.Add(uint64(row))
   233  			}
   234  			n.Close()
   235  		}
   236  		return true
   237  	})
   238  
   239  	new.cnt.Store(chain.cnt.Load())
   240  
   241  	return new
   242  }
   243  
   244  func (chain *DeleteChain) OnReplayNode(deleteNode *DeleteNode) {
   245  	deleteNode.AttachTo(chain)
   246  	switch deleteNode.nt {
   247  	case NT_Persisted:
   248  	case NT_Merge, NT_Normal:
   249  		it := deleteNode.mask.Iterator()
   250  		for it.HasNext() {
   251  			row := it.Next()
   252  			chain.InsertInDeleteView(row, deleteNode)
   253  		}
   254  	}
   255  	chain.AddDeleteCnt(uint32(deleteNode.mask.GetCardinality()))
   256  	chain.insertInMaskByNode(deleteNode)
   257  	chain.mvcc.IncChangeIntentionCnt()
   258  }
   259  
   260  func (chain *DeleteChain) AddMergeNode() txnif.DeleteNode {
   261  	var merged *DeleteNode
   262  	chain.mvcc.RLock()
   263  	chain.LoopChainLocked(func(n *DeleteNode) bool {
   264  		// Already have a latest merged node
   265  		if n.IsMerged() && merged == nil {
   266  			return false
   267  		} else if n.IsMerged() && merged != nil {
   268  			merged.MergeLocked(n)
   269  			return false
   270  		}
   271  		txn := n.GetTxn()
   272  		if txn != nil {
   273  			return true
   274  		}
   275  		if merged == nil {
   276  			merged = NewMergedNode(n.GetCommitTSLocked())
   277  		}
   278  		merged.MergeLocked(n)
   279  		return true
   280  	})
   281  	if merged != nil {
   282  		merged.AttachTo(chain)
   283  	}
   284  	// chain.RUnlock()
   285  	chain.mvcc.RUnlock()
   286  	return merged
   287  }
   288  
   289  // CollectDeletesInRange collects [startTs, endTs)
   290  func (chain *DeleteChain) CollectDeletesInRangeWithLock(
   291  	startTs, endTs types.TS,
   292  	rwlocker *sync.RWMutex,
   293  ) (mask *nulls.Bitmap, err error) {
   294  	for {
   295  		needWaitFound := false
   296  		mask = nil
   297  		chain.LoopChainLocked(func(n *DeleteNode) bool {
   298  			// Merged node is a loop breaker
   299  			if n.IsMerged() {
   300  				commitTS := n.GetCommitTSLocked()
   301  				if commitTS.Greater(&endTs) {
   302  					return true
   303  				}
   304  				if mask == nil {
   305  					mask = nulls.NewWithSize(int(n.mask.Maximum()))
   306  				}
   307  				mergeDelete(mask, n)
   308  				return false
   309  			}
   310  			needWait, txnToWait := n.NeedWaitCommitting(endTs)
   311  			if needWait {
   312  				rwlocker.RUnlock()
   313  				txnToWait.GetTxnState(true)
   314  				rwlocker.RLock()
   315  				needWaitFound = true
   316  				return false
   317  			}
   318  			if n.IsVisibleByTS(endTs) && !n.IsVisibleByTS(startTs) {
   319  				if mask == nil {
   320  					mask = nulls.NewWithSize(int(n.mask.Maximum()))
   321  				}
   322  				mergeDelete(mask, n)
   323  			}
   324  			return true
   325  		})
   326  		if !needWaitFound {
   327  			break
   328  		}
   329  	}
   330  	return
   331  }
   332  
   333  // any uncommited node, return true
   334  // any committed node with prepare ts within [from, to], return true
   335  func (chain *DeleteChain) HasDeleteIntentsPreparedInLocked(from, to types.TS) (found, isPersisted bool) {
   336  	chain.LoopChainLocked(func(n *DeleteNode) bool {
   337  		if n.IsMerged() {
   338  			found, _ = n.PreparedIn(from, to)
   339  			return false
   340  		}
   341  
   342  		if n.IsActive() {
   343  			return true
   344  		}
   345  
   346  		if n.nt == NT_Persisted {
   347  			isPersisted = true
   348  		}
   349  		found, _ = n.PreparedIn(from, to)
   350  		if n.IsAborted() {
   351  			found = false
   352  		}
   353  		return !found
   354  	})
   355  	return
   356  }
   357  
   358  func (chain *DeleteChain) ResetPersistedMask() { chain.persistedMask = &nulls.Bitmap{} }
   359  
   360  func mergeDelete(mask *nulls.Bitmap, node *DeleteNode) {
   361  	if node == nil || node.mask == nil {
   362  		return
   363  	}
   364  	it := node.mask.Iterator()
   365  	for it.HasNext() {
   366  		mask.Add(uint64(it.Next()))
   367  	}
   368  }
   369  
   370  func (chain *DeleteChain) CollectDeletesLocked(
   371  	txn txnif.TxnReader,
   372  	rwlocker *sync.RWMutex) (merged *nulls.Bitmap, err error) {
   373  	for {
   374  		needWaitFound := false
   375  		merged = chain.mask.Clone()
   376  		chain.LoopChainLocked(func(n *DeleteNode) bool {
   377  			needWait, txnToWait := n.NeedWaitCommitting(txn.GetStartTS())
   378  			if needWait {
   379  				rwlocker.RUnlock()
   380  				txnToWait.GetTxnState(true)
   381  				rwlocker.RLock()
   382  				needWaitFound = true
   383  				return false
   384  			}
   385  			if !n.IsVisible(txn) {
   386  				it := n.GetDeleteMaskLocked().Iterator()
   387  				if n.dt != handle.DT_MergeCompact {
   388  					for it.HasNext() {
   389  						row := it.Next()
   390  						merged.Del(uint64(row))
   391  					}
   392  				} else {
   393  					ts := txn.GetStartTS()
   394  					rt := chain.mvcc.meta.GetObjectData().GetRuntime()
   395  					tsMapping := rt.TransferDelsMap.GetDelsForBlk(*objectio.NewBlockidWithObjectID(&chain.mvcc.meta.ID, chain.mvcc.blkID)).Mapping
   396  					if tsMapping == nil {
   397  						logutil.Warnf("flushtabletail check special dels for %s, no tsMapping", chain.mvcc.meta.ID.String())
   398  						return true
   399  					}
   400  					for it.HasNext() {
   401  						row := it.Next()
   402  						committs, ok := tsMapping[int(row)]
   403  						if !ok {
   404  							logutil.Errorf("flushtabletail check Transfer dels for %s row %d not in dels", chain.mvcc.meta.ID.String(), row)
   405  							continue
   406  						}
   407  						// if the ts can't see the del, then remove it from merged
   408  						if committs.Greater(&ts) {
   409  							merged.Del(uint64(row))
   410  						}
   411  					}
   412  				}
   413  			}
   414  			return true
   415  		})
   416  
   417  		if !needWaitFound {
   418  			break
   419  		}
   420  	}
   421  
   422  	return merged, err
   423  }
   424  
   425  func (chain *DeleteChain) GetDeleteNodeByRow(row uint32) (n *DeleteNode) {
   426  	return chain.links[row]
   427  }