github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/tables/base.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 tables 16 17 import ( 18 "bytes" 19 "context" 20 "fmt" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 "github.com/RoaringBitmap/roaring" 26 "github.com/matrixorigin/matrixone/pkg/common/moerr" 27 "github.com/matrixorigin/matrixone/pkg/container/types" 28 "github.com/matrixorigin/matrixone/pkg/objectio" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/buffer/base" 30 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" 31 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 32 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 33 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/dataio/blockio" 34 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/data" 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/model" 38 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables/jobs" 39 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables/updates" 40 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks" 41 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal" 42 ) 43 44 type BlockT[T common.IRef] interface { 45 common.IRef 46 Pin() *common.PinnedItem[T] 47 GetID() *common.ID 48 } 49 50 type baseBlock struct { 51 common.RefHelper 52 *sync.RWMutex 53 bufMgr base.INodeManager 54 fs *objectio.ObjectFS 55 scheduler tasks.TaskScheduler 56 meta *catalog.BlockEntry 57 mvcc *updates.MVCCHandle 58 ttl time.Time 59 impl data.Block 60 61 node atomic.Pointer[Node] 62 } 63 64 func newBaseBlock( 65 impl data.Block, 66 meta *catalog.BlockEntry, 67 bufMgr base.INodeManager, 68 fs *objectio.ObjectFS, 69 scheduler tasks.TaskScheduler) *baseBlock { 70 blk := &baseBlock{ 71 impl: impl, 72 bufMgr: bufMgr, 73 fs: fs, 74 scheduler: scheduler, 75 meta: meta, 76 ttl: time.Now(), 77 } 78 blk.mvcc = updates.NewMVCCHandle(meta) 79 blk.RWMutex = blk.mvcc.RWMutex 80 return blk 81 } 82 83 func (blk *baseBlock) Close() { 84 blk.meta = nil 85 blk.mvcc.Close() 86 blk.mvcc = nil 87 } 88 89 func (blk *baseBlock) PinNode() *Node { 90 n := blk.node.Load() 91 // if ref fails, reload. 92 // Note: avoid bad case where releasing happens before Ref() 93 for ; !n.RefIfHasRef(); n = blk.node.Load() { 94 } 95 return n 96 } 97 98 func (blk *baseBlock) GetColumnData( 99 from uint32, 100 to uint32, 101 colIdx int, 102 buffer *bytes.Buffer) (vec containers.Vector, err error) { 103 node := blk.PinNode() 104 defer node.Unref() 105 if !node.IsPersisted() { 106 blk.RLock() 107 defer blk.RUnlock() 108 return node.GetColumnDataWindow(from, to, colIdx, buffer) 109 } else { 110 return node.GetColumnDataWindow(from, to, colIdx, buffer) 111 } 112 } 113 114 func (blk *baseBlock) Rows() int { 115 node := blk.PinNode() 116 defer node.Unref() 117 if !node.IsPersisted() { 118 blk.RLock() 119 defer blk.RUnlock() 120 return int(node.Rows()) 121 } else { 122 return int(node.Rows()) 123 } 124 } 125 126 func (blk *baseBlock) TryUpgrade() (err error) { 127 node := blk.node.Load() 128 if node.IsPersisted() { 129 return 130 } 131 pnode := newPersistedNode(blk) 132 nnode := NewNode(pnode) 133 nnode.Ref() 134 135 if !blk.node.CompareAndSwap(node, nnode) { 136 nnode.Unref() 137 } else { 138 node.Unref() 139 } 140 return 141 } 142 143 func (blk *baseBlock) GetMeta() any { return blk.meta } 144 func (blk *baseBlock) GetBufMgr() base.INodeManager { return blk.bufMgr } 145 func (blk *baseBlock) GetFs() *objectio.ObjectFS { return blk.fs } 146 func (blk *baseBlock) GetID() *common.ID { return blk.meta.AsCommonID() } 147 148 func (blk *baseBlock) FillInMemoryDeletesLocked( 149 view *model.BaseView, 150 rwlocker *sync.RWMutex) (err error) { 151 chain := blk.mvcc.GetDeleteChain() 152 n, err := chain.CollectDeletesLocked(view.Ts, false, rwlocker) 153 if err != nil { 154 return 155 } 156 dnode := n.(*updates.DeleteNode) 157 if dnode != nil { 158 if view.DeleteMask == nil { 159 view.DeleteMask = dnode.GetDeleteMaskLocked() 160 } else { 161 view.DeleteMask.Or(dnode.GetDeleteMaskLocked()) 162 } 163 } 164 return 165 } 166 167 func (blk *baseBlock) LoadPersistedCommitTS() (vec containers.Vector, err error) { 168 if !blk.meta.IsAppendable() { 169 return 170 } 171 location := blk.meta.GetMetaLoc() 172 if location == "" { 173 return 174 } 175 reader, err := blockio.NewReader(context.Background(), blk.fs, location) 176 if err != nil { 177 return 178 } 179 meta, err := reader.ReadMeta(nil) 180 if err != nil { 181 return 182 } 183 bat, err := reader.LoadBlkColumnsByMetaAndIdx( 184 []types.Type{types.T_TS.ToType()}, 185 []string{catalog.AttrCommitTs}, 186 []bool{false}, 187 meta, 188 len(blk.meta.GetSchema().NameIndex), 189 ) 190 if err != nil { 191 return 192 } 193 vec = bat.Vecs[0] 194 return 195 } 196 197 func (blk *baseBlock) LoadPersistedData() (bat *containers.Batch, err error) { 198 schema := blk.meta.GetSchema() 199 bat = containers.NewBatch() 200 defer func() { 201 if err != nil { 202 bat.Close() 203 } 204 }() 205 206 var vec containers.Vector 207 for i, col := range schema.ColDefs { 208 vec, err = blk.LoadPersistedColumnData(i, nil) 209 if err != nil { 210 return 211 } 212 bat.AddVector(col.Name, vec) 213 } 214 return 215 } 216 217 func (blk *baseBlock) LoadPersistedColumnData( 218 colIdx int, 219 buffer *bytes.Buffer, 220 ) (vec containers.Vector, err error) { 221 def := blk.meta.GetSchema().ColDefs[colIdx] 222 location := blk.meta.GetMetaLoc() 223 return LoadPersistedColumnData( 224 blk.bufMgr, 225 blk.fs, 226 blk.meta.AsCommonID(), 227 def, 228 location, 229 buffer) 230 } 231 232 func (blk *baseBlock) LoadPersistedDeletes() (bat *containers.Batch, err error) { 233 location := blk.meta.GetDeltaLoc() 234 if location == "" { 235 return 236 } 237 return LoadPersistedDeletes( 238 blk.bufMgr, 239 blk.fs, 240 location) 241 } 242 243 func (blk *baseBlock) FillPersistedDeletes( 244 view *model.BaseView) (err error) { 245 deletes, err := blk.LoadPersistedDeletes() 246 if deletes == nil || err != nil { 247 return nil 248 } 249 for i := 0; i < deletes.Length(); i++ { 250 abort := deletes.Vecs[2].Get(i).(bool) 251 if abort { 252 continue 253 } 254 commitTS := deletes.Vecs[1].Get(i).(types.TS) 255 if commitTS.Greater(view.Ts) { 256 continue 257 } 258 rowid := deletes.Vecs[0].Get(i).(types.Rowid) 259 _, _, row := model.DecodePhyAddrKey(rowid) 260 if view.DeleteMask == nil { 261 view.DeleteMask = roaring.NewBitmap() 262 } 263 view.DeleteMask.Add(row) 264 } 265 return nil 266 } 267 268 func (blk *baseBlock) ResolvePersistedColumnDatas( 269 pnode *persistedNode, 270 ts types.TS, 271 colIdxs []int, 272 buffers []*bytes.Buffer, 273 skipDeletes bool) (view *model.BlockView, err error) { 274 data, err := blk.LoadPersistedData() 275 if err != nil { 276 return nil, err 277 } 278 view = model.NewBlockView(ts) 279 for _, colIdx := range colIdxs { 280 view.SetData(colIdx, data.Vecs[colIdx]) 281 } 282 283 if skipDeletes { 284 return 285 } 286 287 defer func() { 288 if err != nil { 289 view.Close() 290 } 291 }() 292 293 if err = blk.FillPersistedDeletes(view.BaseView); err != nil { 294 return 295 } 296 297 blk.RLock() 298 defer blk.RUnlock() 299 err = blk.FillInMemoryDeletesLocked(view.BaseView, blk.RWMutex) 300 return 301 } 302 303 func (blk *baseBlock) ResolvePersistedColumnData( 304 pnode *persistedNode, 305 ts types.TS, 306 colIdx int, 307 buffer *bytes.Buffer, 308 skipDeletes bool) (view *model.ColumnView, err error) { 309 view = model.NewColumnView(ts, colIdx) 310 vec, err := blk.LoadPersistedColumnData(colIdx, buffer) 311 if err != nil { 312 return 313 } 314 view.SetData(vec) 315 316 if skipDeletes { 317 return 318 } 319 320 defer func() { 321 if err != nil { 322 view.Close() 323 } 324 }() 325 326 if err = blk.FillPersistedDeletes(view.BaseView); err != nil { 327 return 328 } 329 330 blk.RLock() 331 defer blk.RUnlock() 332 err = blk.FillInMemoryDeletesLocked(view.BaseView, blk.RWMutex) 333 return 334 } 335 336 func (blk *baseBlock) PersistedBatchDedup( 337 pnode *persistedNode, 338 ts types.TS, 339 keys containers.Vector, 340 rowmask *roaring.Bitmap, 341 dedupClosure func( 342 containers.Vector, 343 types.TS, 344 *roaring.Bitmap, 345 *catalog.ColDef, 346 ) func(any, int) error) (err error) { 347 sels, err := pnode.BatchDedup( 348 keys, 349 nil, 350 ) 351 if err == nil || !moerr.IsMoErrCode(err, moerr.OkExpectedPossibleDup) { 352 return 353 } 354 def := blk.meta.GetSchema().GetSingleSortKey() 355 view, err := blk.ResolvePersistedColumnData( 356 pnode, 357 ts, 358 def.Idx, 359 nil, 360 false) 361 if err != nil { 362 return 363 } 364 if rowmask != nil { 365 if view.DeleteMask == nil { 366 view.DeleteMask = rowmask 367 } else { 368 view.DeleteMask.Or(rowmask) 369 } 370 } 371 defer view.Close() 372 dedupFn := dedupClosure(view.GetData(), ts, view.DeleteMask, def) 373 err = keys.Foreach(dedupFn, sels) 374 return 375 } 376 377 func (blk *baseBlock) getPersistedValue( 378 pnode *persistedNode, 379 ts types.TS, 380 row, col int, 381 skipMemory bool) (v any, err error) { 382 view := model.NewColumnView(ts, col) 383 if err = blk.FillPersistedDeletes(view.BaseView); err != nil { 384 return 385 } 386 if !skipMemory { 387 blk.RLock() 388 err = blk.FillInMemoryDeletesLocked(view.BaseView, blk.RWMutex) 389 blk.RUnlock() 390 if err != nil { 391 return 392 } 393 } 394 if view.DeleteMask != nil && view.DeleteMask.ContainsInt(row) { 395 err = moerr.NewNotFoundNoCtx() 396 return 397 } 398 view2, err := blk.ResolvePersistedColumnData(pnode, ts, col, nil, true) 399 if err != nil { 400 return 401 } 402 defer view2.Close() 403 v = view2.GetValue(row) 404 return 405 } 406 407 func (blk *baseBlock) DeletesInfo() string { 408 blk.RLock() 409 defer blk.RUnlock() 410 return blk.mvcc.GetDeleteChain().StringLocked() 411 } 412 413 func (blk *baseBlock) RangeDelete( 414 txn txnif.AsyncTxn, 415 start, end uint32, 416 dt handle.DeleteType) (node txnif.DeleteNode, err error) { 417 blk.Lock() 418 defer blk.Unlock() 419 if err = blk.mvcc.CheckNotDeleted(start, end, txn.GetStartTS()); err != nil { 420 return 421 } 422 node = blk.mvcc.CreateDeleteNode(txn, dt) 423 node.RangeDeleteLocked(start, end) 424 return 425 } 426 427 func (blk *baseBlock) PPString(level common.PPLevel, depth int, prefix string) string { 428 s := fmt.Sprintf("%s | [Rows=%d]", blk.meta.PPString(level, depth, prefix), blk.Rows()) 429 if level >= common.PPL1 { 430 blk.RLock() 431 s2 := blk.mvcc.StringLocked() 432 blk.RUnlock() 433 if s2 != "" { 434 s = fmt.Sprintf("%s\n%s", s, s2) 435 } 436 } 437 return s 438 } 439 440 func (blk *baseBlock) HasDeleteIntentsPreparedIn(from, to types.TS) (found bool) { 441 blk.RLock() 442 defer blk.RUnlock() 443 found = blk.mvcc.GetDeleteChain().HasDeleteIntentsPreparedInLocked(from, to) 444 return 445 } 446 447 func (blk *baseBlock) CollectAppendLogIndexes(startTs, endTs types.TS) (indexes []*wal.Index, err error) { 448 blk.RLock() 449 defer blk.RUnlock() 450 return blk.mvcc.CollectAppendLogIndexesLocked(startTs, endTs) 451 } 452 453 func (blk *baseBlock) CollectChangesInRange(startTs, endTs types.TS) (view *model.BlockView, err error) { 454 view = model.NewBlockView(endTs) 455 blk.RLock() 456 defer blk.RUnlock() 457 deleteChain := blk.mvcc.GetDeleteChain() 458 view.DeleteMask, view.DeleteLogIndexes, err = 459 deleteChain.CollectDeletesInRange(startTs, endTs, blk.RWMutex) 460 return 461 } 462 463 func (blk *baseBlock) CollectDeleteInRange( 464 start, end types.TS, 465 withAborted bool) (bat *containers.Batch, err error) { 466 rowID, ts, abort, abortedMap := blk.mvcc.CollectDelete(start, end) 467 if rowID == nil { 468 return 469 } 470 bat = containers.NewBatch() 471 bat.AddVector(catalog.PhyAddrColumnName, rowID) 472 bat.AddVector(catalog.AttrCommitTs, ts) 473 if withAborted { 474 bat.AddVector(catalog.AttrAborted, abort) 475 } else { 476 bat.Deletes = abortedMap 477 bat.Compact() 478 } 479 return 480 } 481 482 func (blk *baseBlock) adjustScore( 483 rawScoreFn func() (int, bool), 484 ttl time.Duration, 485 force bool) int { 486 score, dropped := rawScoreFn() 487 if dropped { 488 return 0 489 } 490 if force { 491 score = 100 492 } 493 if score == 0 || score > 1 { 494 return score 495 } 496 var ratio float32 497 if blk.meta.IsAppendable() { 498 currRows := uint32(blk.Rows()) 499 ratio = float32(currRows) / float32(blk.meta.GetSchema().BlockMaxRows) 500 if ratio >= 0 && ratio < 0.2 { 501 ttl = 3*ttl - ttl/2 502 } else if ratio >= 0.2 && ratio < 0.4 { 503 ttl = 2 * ttl 504 } else if ratio >= 0.4 && ratio < 0.6 { 505 ttl = 2*ttl - ttl/2 506 } 507 } 508 509 deleteCnt := blk.mvcc.GetDeleteCnt() 510 ratio = float32(deleteCnt) / float32(blk.meta.GetSchema().BlockMaxRows) 511 if ratio <= 1 && ratio > 0.5 { 512 ttl /= 10 513 } else if ratio <= 0.5 && ratio > 0.3 { 514 ttl /= 5 515 } else if ratio <= 0.3 && ratio > 0.2 { 516 ttl /= 3 517 } else if ratio <= 0.2 && ratio > 0.1 { 518 ttl /= 2 519 } 520 521 if time.Now().After(blk.ttl.Add(ttl)) { 522 return 100 523 } 524 return 1 525 } 526 527 func (blk *baseBlock) OnReplayDelete(node txnif.DeleteNode) (err error) { 528 blk.mvcc.OnReplayDeleteNode(node) 529 err = node.OnApply() 530 return 531 } 532 533 func (blk *baseBlock) OnReplayAppend(_ txnif.AppendNode) (err error) { 534 panic("not supported") 535 } 536 537 func (blk *baseBlock) OnReplayAppendPayload(_ *containers.Batch) (err error) { 538 panic("not supported") 539 } 540 541 func (blk *baseBlock) MakeAppender() (appender data.BlockAppender, err error) { 542 panic("not supported") 543 } 544 545 func (blk *baseBlock) GetRowsOnReplay() uint64 { 546 rows := uint64(blk.mvcc.GetTotalRow()) 547 metaLoc := blk.meta.GetMetaLoc() 548 if metaLoc == "" { 549 return rows 550 } 551 meta, err := blockio.DecodeMetaLocToMeta(metaLoc) 552 if err != nil { 553 panic(err) 554 } 555 fileRows := uint64(meta.GetRows()) 556 if rows > fileRows { 557 return rows 558 } 559 return fileRows 560 } 561 562 func (blk *baseBlock) GetTotalChanges() int { 563 return int(blk.mvcc.GetChangeNodeCnt()) 564 } 565 566 func (blk *baseBlock) IsAppendable() bool { return false } 567 568 func (blk *baseBlock) MutationInfo() string { 569 rows := blk.Rows() 570 totalChanges := blk.mvcc.GetChangeNodeCnt() 571 s := fmt.Sprintf("Block %s Mutation Info: Changes=%d/%d", 572 blk.meta.AsCommonID().BlockString(), 573 totalChanges, 574 rows) 575 if totalChanges == 0 { 576 return s 577 } 578 deleteCnt := blk.mvcc.GetDeleteCnt() 579 if deleteCnt != 0 { 580 s = fmt.Sprintf("%s, Del:%d/%d", s, deleteCnt, rows) 581 } 582 return s 583 } 584 585 func (blk *baseBlock) BuildCompactionTaskFactory() ( 586 factory tasks.TxnTaskFactory, 587 taskType tasks.TaskType, 588 scopes []common.ID, 589 err error) { 590 591 if !blk.impl.PrepareCompact() { 592 return 593 } 594 595 factory = jobs.CompactBlockTaskFactory(blk.meta, blk.scheduler) 596 taskType = tasks.DataCompactionTask 597 scopes = append(scopes, *blk.meta.AsCommonID()) 598 return 599 } 600 601 func (blk *baseBlock) CollectAppendInRange(start, end types.TS, withAborted bool) (*containers.Batch, error) { 602 return nil, nil 603 }