github.com/matrixorigin/matrixone@v0.7.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/types" 23 24 "github.com/RoaringBitmap/roaring" 25 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/handle" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal" 30 ) 31 32 type DeleteChain struct { 33 *sync.RWMutex 34 *txnbase.MVCCChain 35 mvcc *MVCCHandle 36 links map[uint32]*common.GenericSortedDList[txnif.MVCCNode] 37 cnt atomic.Uint32 38 } 39 40 func NewDeleteChain(rwlocker *sync.RWMutex, mvcc *MVCCHandle) *DeleteChain { 41 if rwlocker == nil { 42 rwlocker = new(sync.RWMutex) 43 } 44 chain := &DeleteChain{ 45 RWMutex: rwlocker, 46 MVCCChain: txnbase.NewMVCCChain(compareDeleteNode, NewEmptyDeleteNode), 47 links: make(map[uint32]*common.GenericSortedDList[txnif.MVCCNode]), 48 mvcc: mvcc, 49 } 50 return chain 51 } 52 func (chain *DeleteChain) Close() { 53 chain.mvcc = nil 54 } 55 func (chain *DeleteChain) AddDeleteCnt(cnt uint32) { 56 chain.cnt.Add(cnt) 57 } 58 59 func (chain *DeleteChain) GetDeleteCnt() uint32 { 60 return chain.cnt.Load() 61 } 62 63 func (chain *DeleteChain) StringLocked() string { 64 msg := "DeleteChain:" 65 line := 1 66 chain.LoopChain(func(vn txnif.MVCCNode) bool { 67 n := vn.(*DeleteNode) 68 msg = fmt.Sprintf("%s\n%d. %s", msg, line, n.StringLocked()) 69 line++ 70 return true 71 }) 72 return msg 73 } 74 75 func (chain *DeleteChain) GetController() *MVCCHandle { return chain.mvcc } 76 77 func (chain *DeleteChain) IsDeleted(row uint32, ts types.TS, rwlocker *sync.RWMutex) (deleted bool, err error) { 78 deleteNode := chain.GetDeleteNodeByRow(row) 79 if deleteNode == nil { 80 return false, nil 81 } 82 needWait, txn := deleteNode.NeedWaitCommitting(ts) 83 if needWait { 84 rwlocker.RUnlock() 85 txn.GetTxnState(true) 86 rwlocker.RLock() 87 } 88 return deleteNode.IsVisible(ts), nil 89 } 90 91 func (chain *DeleteChain) PrepareRangeDelete(start, end uint32, ts types.TS) (err error) { 92 chain.LoopChain( 93 func(vn txnif.MVCCNode) bool { 94 n := vn.(*DeleteNode) 95 overlap := n.HasOverlapLocked(start, end) 96 if overlap { 97 err = txnif.ErrTxnWWConflict 98 return false 99 } 100 return true 101 }) 102 return 103 } 104 105 func (chain *DeleteChain) UpdateLocked(node *DeleteNode) { 106 chain.MVCC.Update(node.GenericDLNode) 107 } 108 109 func (chain *DeleteChain) RemoveNodeLocked(node txnif.DeleteNode) { 110 chain.MVCC.Delete(node.(*DeleteNode).GenericDLNode) 111 } 112 113 func (chain *DeleteChain) DepthLocked() int { return chain.MVCC.Depth() } 114 115 func (chain *DeleteChain) AddNodeLocked(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode { 116 node := NewDeleteNode(txn, deleteType) 117 node.AttachTo(chain) 118 return node 119 } 120 func (chain *DeleteChain) InsertInDeleteView(row uint32, deleteNode *DeleteNode) { 121 var link *common.GenericSortedDList[txnif.MVCCNode] 122 if link = chain.links[row]; link == nil { 123 link = common.NewGenericSortedDList(compareDeleteNode) 124 n := link.Insert(deleteNode) 125 deleteNode.viewNodes[row] = n 126 chain.links[row] = link 127 return 128 } 129 link.Insert(deleteNode) 130 } 131 func (chain *DeleteChain) DeleteInDeleteView(deleteNode *DeleteNode) { 132 it := deleteNode.mask.Iterator() 133 for it.HasNext() { 134 row := it.Next() 135 link := chain.links[row] 136 link.Delete(deleteNode.viewNodes[row]) 137 if link.Depth() == 0 { 138 delete(chain.links, row) 139 } 140 } 141 } 142 func (chain *DeleteChain) OnReplayNode(deleteNode *DeleteNode) { 143 it := deleteNode.mask.Iterator() 144 for it.HasNext() { 145 row := it.Next() 146 chain.InsertInDeleteView(row, deleteNode) 147 } 148 deleteNode.AttachTo(chain) 149 chain.AddDeleteCnt(uint32(deleteNode.mask.GetCardinality())) 150 chain.mvcc.IncChangeNodeCnt() 151 } 152 153 func (chain *DeleteChain) AddMergeNode() txnif.DeleteNode { 154 var merged *DeleteNode 155 chain.mvcc.RLock() 156 // chain.RLock() 157 chain.LoopChain(func(vn txnif.MVCCNode) bool { 158 n := vn.(*DeleteNode) 159 // Already have a latest merged node 160 if n.IsMerged() && merged == nil { 161 return false 162 } else if n.IsMerged() && merged != nil { 163 merged.MergeLocked(n, true) 164 return false 165 } 166 txn := n.GetTxn() 167 if txn != nil { 168 return true 169 } 170 if merged == nil { 171 merged = NewMergedNode(n.GetCommitTSLocked()) 172 } 173 merged.MergeLocked(n, true) 174 return true 175 }) 176 if merged != nil { 177 merged.AttachTo(chain) 178 } 179 // chain.RUnlock() 180 chain.mvcc.RUnlock() 181 return merged 182 } 183 184 // CollectDeletesInRange collects [startTs, endTs) 185 func (chain *DeleteChain) CollectDeletesInRange( 186 startTs, endTs types.TS, 187 rwlocker *sync.RWMutex) (mask *roaring.Bitmap, indexes []*wal.Index, err error) { 188 n, err := chain.CollectDeletesLocked(startTs, true, rwlocker) 189 if err != nil { 190 return 191 } 192 startNode := n.(*DeleteNode) 193 // n, err = chain.CollectDeletesLocked(endTs-1, true) 194 n, err = chain.CollectDeletesLocked(endTs, true, rwlocker) 195 if err != nil { 196 return 197 } 198 endNode := n.(*DeleteNode) 199 if endNode == nil { 200 return 201 } 202 if startNode == nil { 203 mask = endNode.GetDeleteMaskLocked() 204 indexes = endNode.logIndexes 205 return 206 } 207 mask = endNode.GetDeleteMaskLocked() 208 mask2 := startNode.GetDeleteMaskLocked() 209 mask.AndNot(mask2) 210 indexes = endNode.logIndexes[len(startNode.logIndexes):] 211 return 212 } 213 214 // any uncommited node, return true 215 // any committed node with prepare ts within [from, to], return true 216 func (chain *DeleteChain) HasDeleteIntentsPreparedInLocked(from, to types.TS) (found bool) { 217 chain.LoopChain(func(vn txnif.MVCCNode) bool { 218 n := vn.(*DeleteNode) 219 if n.IsMerged() { 220 found, _ = n.PreparedIn(from, to) 221 return false 222 } 223 224 if n.IsActive() { 225 return true 226 } 227 228 found, _ = n.PreparedIn(from, to) 229 if n.IsAborted() { 230 found = false 231 } 232 return !found 233 }) 234 return 235 } 236 237 func (chain *DeleteChain) CollectDeletesLocked( 238 ts types.TS, 239 collectIndex bool, 240 rwlocker *sync.RWMutex) (txnif.DeleteNode, error) { 241 var merged *DeleteNode 242 var err error 243 chain.LoopChain(func(vn txnif.MVCCNode) bool { 244 n := vn.(*DeleteNode) 245 // Merged node is a loop breaker 246 if n.IsMerged() { 247 if n.GetCommitTSLocked().Greater(ts) { 248 return true 249 } 250 if merged == nil { 251 merged = NewMergedNode(n.GetCommitTSLocked()) 252 } 253 merged.MergeLocked(n, collectIndex) 254 return false 255 } 256 needWait, txnToWait := n.NeedWaitCommitting(ts) 257 if needWait { 258 rwlocker.RUnlock() 259 txnToWait.GetTxnState(true) 260 rwlocker.RLock() 261 } 262 if n.IsVisible(ts) { 263 if merged == nil { 264 merged = NewMergedNode(n.GetCommitTSLocked()) 265 } 266 merged.MergeLocked(n, collectIndex) 267 } 268 return true 269 }) 270 return merged, err 271 } 272 273 func (chain *DeleteChain) GetDeleteNodeByRow(row uint32) (n *DeleteNode) { 274 link := chain.links[row] 275 if link == nil { 276 return 277 } 278 link.Loop(func(vn *common.GenericDLNode[txnif.MVCCNode]) bool { 279 n = vn.GetPayload().(*DeleteNode) 280 return n.Aborted 281 }, false) 282 return 283 }