github.com/matrixorigin/matrixone@v0.7.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 "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/catalog" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 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/model" 31 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnbase" 32 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal" 33 ) 34 35 type MVCCHandle struct { 36 *sync.RWMutex 37 deletes *DeleteChain 38 meta *catalog.BlockEntry 39 appends *txnbase.MVCCSlice 40 changes atomic.Uint32 41 deletesListener func(uint64, common.RowGen, types.TS) error 42 appendListener func(txnif.AppendNode) error 43 } 44 45 func NewMVCCHandle(meta *catalog.BlockEntry) *MVCCHandle { 46 node := &MVCCHandle{ 47 RWMutex: new(sync.RWMutex), 48 meta: meta, 49 appends: txnbase.NewMVCCSlice(NewEmptyAppendNode, CompareAppendNode), 50 } 51 node.deletes = NewDeleteChain(nil, node) 52 if meta == nil { 53 return node 54 } 55 return node 56 } 57 func (n *MVCCHandle) Close() { 58 n.deletes.Close() 59 n.appends.Close() 60 n.meta = nil 61 } 62 func (n *MVCCHandle) SetAppendListener(l func(txnif.AppendNode) error) { 63 n.appendListener = l 64 } 65 66 func (n *MVCCHandle) GetAppendListener() func(txnif.AppendNode) error { 67 return n.appendListener 68 } 69 70 func (n *MVCCHandle) SetDeletesListener(l func(uint64, common.RowGen, types.TS) error) { 71 n.deletesListener = l 72 } 73 74 func (n *MVCCHandle) GetDeletesListener() func(uint64, common.RowGen, types.TS) error { 75 return n.deletesListener 76 } 77 78 func (n *MVCCHandle) HasActiveAppendNode() bool { 79 n.RLock() 80 defer n.RUnlock() 81 return !n.appends.IsCommitted() 82 } 83 84 func (n *MVCCHandle) IncChangeNodeCnt() { 85 n.changes.Add(1) 86 } 87 88 func (n *MVCCHandle) GetChangeNodeCnt() uint32 { 89 return n.changes.Load() 90 } 91 92 func (n *MVCCHandle) GetDeleteCnt() uint32 { 93 return n.deletes.GetDeleteCnt() 94 } 95 96 func (n *MVCCHandle) GetID() *common.ID { return n.meta.AsCommonID() } 97 func (n *MVCCHandle) GetEntry() *catalog.BlockEntry { return n.meta } 98 99 func (n *MVCCHandle) StringLocked() string { 100 s := "" 101 if n.deletes.DepthLocked() > 0 { 102 s = fmt.Sprintf("%s%s", s, n.deletes.StringLocked()) 103 } 104 s = fmt.Sprintf("%s\n%s", s, n.appends.StringLocked()) 105 return s 106 } 107 108 func (n *MVCCHandle) CheckNotDeleted(start, end uint32, ts types.TS) error { 109 return n.deletes.PrepareRangeDelete(start, end, ts) 110 } 111 112 func (n *MVCCHandle) CreateDeleteNode(txn txnif.AsyncTxn, deleteType handle.DeleteType) txnif.DeleteNode { 113 return n.deletes.AddNodeLocked(txn, deleteType) 114 } 115 116 func (n *MVCCHandle) OnReplayDeleteNode(deleteNode txnif.DeleteNode) { 117 n.deletes.OnReplayNode(deleteNode.(*DeleteNode)) 118 } 119 120 func (n *MVCCHandle) GetDeleteChain() *DeleteChain { 121 return n.deletes 122 } 123 func (n *MVCCHandle) OnReplayAppendNode(an *AppendNode) { 124 an.mvcc = n 125 n.appends.InsertNode(an) 126 } 127 func (n *MVCCHandle) AddAppendNodeLocked( 128 txn txnif.AsyncTxn, 129 startRow uint32, 130 maxRow uint32) (an *AppendNode, created bool) { 131 var ts types.TS 132 if txn != nil { 133 ts = txn.GetStartTS() 134 } 135 if n.appends.IsEmpty() || !n.appends.GetUpdateNodeLocked().(*AppendNode).Start.Equal(ts) { 136 an = NewAppendNode(txn, startRow, maxRow, n) 137 n.appends.InsertNode(an) 138 created = true 139 } else { 140 an = n.appends.GetUpdateNodeLocked().(*AppendNode) 141 created = false 142 an.SetMaxRow(maxRow) 143 } 144 return 145 } 146 func (n *MVCCHandle) AppendCommitted() bool { 147 return n.appends.IsCommitted() 148 } 149 func (n *MVCCHandle) DeleteAppendNodeLocked(node *AppendNode) { 150 n.appends.DeleteNode(node) 151 } 152 153 func (n *MVCCHandle) IsVisibleLocked(row uint32, ts types.TS) (bool, error) { 154 an := n.GetAppendNodeByRow(row) 155 return an.IsVisible(ts), nil 156 } 157 158 func (n *MVCCHandle) IsDeletedLocked(row uint32, ts types.TS, rwlocker *sync.RWMutex) (bool, error) { 159 return n.deletes.IsDeleted(row, ts, rwlocker) 160 } 161 162 // 1 2 3 4 5 6 163 // [----] [---------] [----][------][-----] [-----] 164 // 165 // -----------+------------------+----------------------> 166 // 167 // start end 168 func (n *MVCCHandle) CollectUncommittedANodesPreparedBefore( 169 ts types.TS, 170 fn func(*AppendNode)) (anyWaitable bool) { 171 if n.appends.IsEmpty() { 172 return 173 } 174 n.appends.ForEach(func(un txnif.MVCCNode) bool { 175 an := un.(*AppendNode) 176 needWait, txn := an.NeedWaitCommitting(ts) 177 if txn == nil { 178 return false 179 } 180 if needWait { 181 fn(an) 182 anyWaitable = true 183 } 184 return true 185 }, false) 186 return 187 } 188 189 func (n *MVCCHandle) CollectAppendLogIndexesLocked(startTs, endTs types.TS) (indexes []*wal.Index, err error) { 190 if n.appends.IsEmpty() { 191 return 192 } 193 indexes = make([]*wal.Index, 0) 194 n.appends.ForEach(func(un txnif.MVCCNode) bool { 195 an := un.(*AppendNode) 196 needWait, txn := an.NeedWaitCommitting(endTs.Next()) 197 if needWait { 198 n.RUnlock() 199 txn.GetTxnState(true) 200 n.RLock() 201 } 202 if an.Prepare.Less(startTs) { 203 return true 204 } 205 if an.Prepare.Greater(endTs) { 206 return false 207 } 208 indexes = append(indexes, an.GetLogIndex()) 209 return true 210 }, true) 211 return 212 } 213 214 func (n *MVCCHandle) GetVisibleRowLocked(ts types.TS) (maxrow uint32, visible bool, holes *roaring.Bitmap, err error) { 215 anToWait := make([]*AppendNode, 0) 216 txnToWait := make([]txnif.TxnReader, 0) 217 n.appends.ForEach(func(un txnif.MVCCNode) bool { 218 an := un.(*AppendNode) 219 needWait, txn := an.NeedWaitCommitting(ts) 220 if needWait { 221 anToWait = append(anToWait, an) 222 txnToWait = append(txnToWait, txn) 223 return true 224 } 225 if an.IsVisible(ts) { 226 visible = true 227 maxrow = an.maxRow 228 } else { 229 if holes == nil { 230 holes = roaring.NewBitmap() 231 } 232 holes.AddRange(uint64(an.startRow), uint64(an.maxRow)) 233 } 234 return !an.Prepare.Greater(ts) 235 }, true) 236 if len(anToWait) != 0 { 237 n.RUnlock() 238 for _, txn := range txnToWait { 239 txn.GetTxnState(true) 240 } 241 n.RLock() 242 } 243 for _, an := range anToWait { 244 if an.IsVisible(ts) { 245 visible = true 246 if maxrow < an.maxRow { 247 maxrow = an.maxRow 248 } 249 } else { 250 if holes == nil { 251 holes = roaring.NewBitmap() 252 } 253 holes.AddRange(uint64(an.startRow), uint64(an.maxRow)) 254 } 255 } 256 if holes != nil { 257 holes.RemoveRange(uint64(maxrow), uint64(holes.Maximum())+1) 258 } 259 return 260 } 261 262 // GetTotalRow is only for replay 263 func (n *MVCCHandle) GetTotalRow() uint32 { 264 van := n.appends.GetUpdateNodeLocked() 265 if van == nil { 266 return 0 267 } 268 an := van.(*AppendNode) 269 return an.maxRow - n.deletes.cnt.Load() 270 } 271 272 func (n *MVCCHandle) CollectAppendLocked( 273 start, end types.TS) ( 274 minRow, maxRow uint32, 275 commitTSVec, abortVec containers.Vector, 276 abortedBitmap *roaring.Bitmap) { 277 startOffset, node := n.appends.GetNodeToReadByPrepareTS(start) 278 if node != nil && node.GetPrepare().Less(start) { 279 startOffset++ 280 } 281 endOffset, node := n.appends.GetNodeToReadByPrepareTS(end) 282 if node == nil || startOffset > endOffset { 283 return 284 } 285 minRow = n.appends.GetNodeByOffset(startOffset).(*AppendNode).startRow 286 maxRow = node.(*AppendNode).maxRow 287 288 abortedBitmap = roaring.NewBitmap() 289 commitTSVec = containers.MakeVector(types.T_TS.ToType(), false) 290 abortVec = containers.MakeVector(types.T_bool.ToType(), false) 291 n.appends.LoopOffsetRange( 292 startOffset, 293 endOffset, 294 func(m txnif.MVCCNode) bool { 295 node := m.(*AppendNode) 296 txn := node.GetTxn() 297 if txn != nil { 298 n.RUnlock() 299 txn.GetTxnState(true) 300 n.RLock() 301 } 302 if node.IsAborted() { 303 abortedBitmap.AddRange(uint64(node.startRow), uint64(node.maxRow)) 304 } 305 for i := 0; i < int(node.maxRow-node.startRow); i++ { 306 commitTSVec.Append(node.GetCommitTS()) 307 abortVec.Append(node.IsAborted()) 308 } 309 return true 310 }) 311 return 312 } 313 314 func (n *MVCCHandle) CollectDelete(start, end types.TS) (rowIDVec, commitTSVec, abortVec containers.Vector, abortedBitmap *roaring.Bitmap) { 315 n.RLock() 316 defer n.RUnlock() 317 if n.deletes.IsEmpty() { 318 return 319 } 320 if !n.ExistDeleteInRange(start, end) { 321 return 322 } 323 324 rowIDVec = containers.MakeVector(types.T_Rowid.ToType(), false) 325 commitTSVec = containers.MakeVector(types.T_TS.ToType(), false) 326 abortVec = containers.MakeVector(types.T_bool.ToType(), false) 327 abortedBitmap = roaring.NewBitmap() 328 prefix := n.meta.MakeKey() 329 330 n.deletes.LoopChain( 331 func(m txnif.MVCCNode) bool { 332 node := m.(*DeleteNode) 333 needWait, txn := node.NeedWaitCommitting(end.Next()) 334 if needWait { 335 n.RUnlock() 336 txn.GetTxnState(true) 337 n.RLock() 338 } 339 in, before := node.PreparedIn(start, end) 340 if in { 341 it := node.mask.Iterator() 342 if node.IsAborted() { 343 abortedBitmap.AddMany(node.mask.ToArray()) 344 } 345 for it.HasNext() { 346 row := it.Next() 347 rowIDVec.Append(model.EncodePhyAddrKeyWithPrefix(prefix, row)) 348 commitTSVec.Append(node.GetEnd()) 349 abortVec.Append(node.IsAborted()) 350 } 351 } 352 return !before 353 }) 354 return 355 } 356 357 func (n *MVCCHandle) ExistDeleteInRange(start, end types.TS) (exist bool) { 358 n.deletes.LoopChain( 359 func(m txnif.MVCCNode) bool { 360 node := m.(*DeleteNode) 361 needWait, txn := node.NeedWaitCommitting(end.Next()) 362 if needWait { 363 n.RUnlock() 364 txn.GetTxnState(true) 365 n.RLock() 366 } 367 in, before := node.PreparedIn(start, end) 368 if in { 369 exist = true 370 return false 371 } 372 return !before 373 }) 374 return 375 } 376 377 func (n *MVCCHandle) GetAppendNodeByRow(row uint32) (an *AppendNode) { 378 _, van := n.appends.SearchNodeByCompareFn(func(a txnif.MVCCNode) int { 379 node := a.(*AppendNode) 380 if node.maxRow <= row { 381 return -1 382 } 383 if node.startRow > row { 384 return 1 385 } 386 return 0 387 }) 388 if van == nil { 389 return nil 390 } 391 return van.(*AppendNode) 392 } 393 func (n *MVCCHandle) GetDeleteNodeByRow(row uint32) (an *DeleteNode) { 394 return n.deletes.GetDeleteNodeByRow(row) 395 }