github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/tables/mnode.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 "context" 19 20 "github.com/RoaringBitmap/roaring" 21 "github.com/matrixorigin/matrixone/pkg/common/moerr" 22 "github.com/matrixorigin/matrixone/pkg/common/mpool" 23 "github.com/matrixorigin/matrixone/pkg/container/types" 24 "github.com/matrixorigin/matrixone/pkg/container/vector" 25 "github.com/matrixorigin/matrixone/pkg/logutil" 26 "github.com/matrixorigin/matrixone/pkg/objectio" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 30 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/handle" 31 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/txnif" 32 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/index" 33 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/index/indexwrapper" 34 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables/updates" 35 ) 36 37 var _ NodeT = (*memoryNode)(nil) 38 39 type memoryNode struct { 40 common.RefHelper 41 object *baseObject 42 writeSchema *catalog.Schema 43 data *containers.Batch 44 45 //index for primary key : Art tree + ZoneMap. 46 pkIndex *indexwrapper.MutIndex 47 } 48 49 func newMemoryNode(object *baseObject) *memoryNode { 50 impl := new(memoryNode) 51 impl.object = object 52 53 // Get the lastest schema, it will not be modified, so just keep the pointer 54 schema := object.meta.GetSchemaLocked() 55 impl.writeSchema = schema 56 // impl.data = containers.BuildBatchWithPool( 57 // schema.AllNames(), schema.AllTypes(), 0, object.rt.VectorPool.Memtable, 58 // ) 59 impl.initPKIndex(schema) 60 impl.OnZeroCB = impl.close 61 return impl 62 } 63 64 func (node *memoryNode) mustData() *containers.Batch { 65 if node.data != nil { 66 return node.data 67 } 68 schema := node.writeSchema 69 opts := containers.Options{ 70 Allocator: common.MutMemAllocator, 71 } 72 node.data = containers.BuildBatch( 73 schema.AllNames(), schema.AllTypes(), opts, 74 ) 75 return node.data 76 } 77 78 func (node *memoryNode) initPKIndex(schema *catalog.Schema) { 79 if !schema.HasPK() { 80 return 81 } 82 pkDef := schema.GetSingleSortKey() 83 node.pkIndex = indexwrapper.NewMutIndex(pkDef.Type) 84 } 85 86 func (node *memoryNode) close() { 87 mvcc := node.object.appendMVCC 88 logutil.Debugf("Releasing Memorynode BLK-%s", node.object.meta.ID.String()) 89 if node.data != nil { 90 node.data.Close() 91 node.data = nil 92 } 93 if node.pkIndex != nil { 94 node.pkIndex.Close() 95 node.pkIndex = nil 96 } 97 node.object = nil 98 mvcc.ReleaseAppends() 99 } 100 101 func (node *memoryNode) IsPersisted() bool { return false } 102 103 func (node *memoryNode) doBatchDedup( 104 ctx context.Context, 105 keys containers.Vector, 106 keysZM index.ZM, 107 skipFn func(row uint32) error, 108 bf objectio.BloomFilter, 109 ) (sels *roaring.Bitmap, err error) { 110 return node.pkIndex.BatchDedup(ctx, keys.GetDownstreamVector(), keysZM, skipFn, bf) 111 } 112 113 func (node *memoryNode) ContainsKey(ctx context.Context, key any, _ uint32) (ok bool, err error) { 114 if err = node.pkIndex.Dedup(ctx, key, nil); err != nil { 115 return 116 } 117 if !moerr.IsMoErrCode(err, moerr.OkExpectedPossibleDup) { 118 return 119 } 120 ok = true 121 err = nil 122 return 123 } 124 125 func (node *memoryNode) GetValueByRow(readSchema *catalog.Schema, row, col int) (v any, isNull bool) { 126 idx, ok := node.writeSchema.SeqnumMap[readSchema.ColDefs[col].SeqNum] 127 if !ok { 128 // TODO(aptend): use default value 129 return nil, true 130 } 131 data := node.mustData() 132 vec := data.Vecs[idx] 133 return vec.Get(row), vec.IsNull(row) 134 } 135 136 func (node *memoryNode) Foreach( 137 readSchema *catalog.Schema, 138 blkID uint16, 139 colIdx int, 140 op func(v any, isNull bool, row int) error, 141 sels []uint32, 142 mp *mpool.MPool, 143 ) error { 144 if node.data == nil { 145 return nil 146 } 147 idx, ok := node.writeSchema.SeqnumMap[readSchema.ColDefs[colIdx].SeqNum] 148 if !ok { 149 v := containers.NewConstNullVector(readSchema.ColDefs[colIdx].Type, int(node.data.Length()), mp) 150 for _, row := range sels { 151 val := v.Get(int(row)) 152 isNull := v.IsNull(int(row)) 153 err := op(val, isNull, int(row)) 154 if err != nil { 155 return err 156 } 157 } 158 return nil 159 } 160 for _, row := range sels { 161 val := node.data.Vecs[idx].Get(int(row)) 162 isNull := node.data.Vecs[idx].IsNull(int(row)) 163 err := op(val, isNull, int(row)) 164 if err != nil { 165 return err 166 } 167 } 168 return nil 169 } 170 171 func (node *memoryNode) GetRowsByKey(key any) (rows []uint32, err error) { 172 return node.pkIndex.GetActiveRow(key) 173 } 174 175 func (node *memoryNode) Rows() (uint32, error) { 176 if node.data == nil { 177 return 0, nil 178 } 179 return uint32(node.data.Length()), nil 180 } 181 182 func (node *memoryNode) EstimateMemSize() int { 183 if node.data == nil { 184 return 0 185 } 186 return node.data.ApproxSize() 187 } 188 189 func (node *memoryNode) GetColumnDataWindow( 190 readSchema *catalog.Schema, 191 from uint32, 192 to uint32, 193 col int, 194 mp *mpool.MPool, 195 ) (vec containers.Vector, err error) { 196 idx, ok := node.writeSchema.SeqnumMap[readSchema.ColDefs[col].SeqNum] 197 if !ok { 198 return containers.NewConstNullVector(readSchema.ColDefs[col].Type, int(to-from), mp), nil 199 } 200 if node.data == nil { 201 vec = containers.MakeVector(node.writeSchema.AllTypes()[idx], mp) 202 return 203 } 204 data := node.data.Vecs[idx] 205 vec = data.CloneWindowWithPool(int(from), int(to-from), node.object.rt.VectorPool.Transient) 206 // vec = data.CloneWindow(int(from), int(to-from), common.MutMemAllocator) 207 return 208 } 209 210 func (node *memoryNode) GetDataWindowOnWriteSchema( 211 from, to uint32, mp *mpool.MPool, 212 ) (bat *containers.BatchWithVersion, err error) { 213 if node.data == nil { 214 schema := node.writeSchema 215 opts := containers.Options{ 216 Allocator: mp, 217 } 218 inner := containers.BuildBatch( 219 schema.AllNames(), schema.AllTypes(), opts, 220 ) 221 return &containers.BatchWithVersion{ 222 Version: node.writeSchema.Version, 223 NextSeqnum: uint16(node.writeSchema.Extra.NextColSeqnum), 224 Seqnums: node.writeSchema.AllSeqnums(), 225 Batch: inner, 226 }, nil 227 } 228 inner := node.data.CloneWindowWithPool(int(from), int(to-from), node.object.rt.VectorPool.Transient) 229 // inner := node.data.CloneWindow(int(from), int(to-from), common.MutMemAllocator) 230 bat = &containers.BatchWithVersion{ 231 Version: node.writeSchema.Version, 232 NextSeqnum: uint16(node.writeSchema.Extra.NextColSeqnum), 233 Seqnums: node.writeSchema.AllSeqnums(), 234 Batch: inner, 235 } 236 return 237 } 238 239 func (node *memoryNode) GetDataWindow( 240 readSchema *catalog.Schema, 241 colIdxes []int, 242 from, to uint32, 243 mp *mpool.MPool, 244 ) (bat *containers.Batch, err error) { 245 if node.data == nil { 246 schema := node.writeSchema 247 opts := containers.Options{ 248 Allocator: mp, 249 } 250 bat = containers.BuildBatch( 251 schema.AllNames(), schema.AllTypes(), opts, 252 ) 253 return 254 } 255 256 // manually clone data 257 bat = containers.NewBatchWithCapacity(len(colIdxes)) 258 if node.data.Deletes != nil { 259 bat.Deletes = bat.WindowDeletes(int(from), int(to-from), false) 260 } 261 for _, colIdx := range colIdxes { 262 colDef := readSchema.ColDefs[colIdx] 263 idx, ok := node.writeSchema.SeqnumMap[colDef.SeqNum] 264 var vec containers.Vector 265 if !ok { 266 vec = containers.NewConstNullVector(colDef.Type, int(to-from), mp) 267 } else { 268 vec = node.data.Vecs[idx].CloneWindowWithPool(int(from), int(to-from), node.object.rt.VectorPool.Transient) 269 } 270 bat.AddVector(colDef.Name, vec) 271 } 272 return 273 } 274 275 func (node *memoryNode) PrepareAppend(rows uint32) (n uint32, err error) { 276 var length uint32 277 if node.data == nil { 278 length = 0 279 } else { 280 length = uint32(node.data.Length()) 281 } 282 283 left := node.writeSchema.BlockMaxRows - length 284 285 if left == 0 { 286 err = moerr.NewInternalErrorNoCtx("not appendable") 287 return 288 } 289 if rows > left { 290 n = left 291 } else { 292 n = rows 293 } 294 return 295 } 296 297 func (node *memoryNode) FillPhyAddrColumn(startRow, length uint32) (err error) { 298 var col *vector.Vector 299 if col, err = objectio.ConstructRowidColumn( 300 objectio.NewBlockidWithObjectID(&node.object.meta.ID, 0), 301 startRow, 302 length, 303 common.MutMemAllocator, 304 ); err != nil { 305 return 306 } 307 err = node.mustData().Vecs[node.writeSchema.PhyAddrKey.Idx].ExtendVec(col) 308 col.Free(common.MutMemAllocator) 309 return 310 } 311 312 func (node *memoryNode) ApplyAppend( 313 bat *containers.Batch, 314 txn txnif.AsyncTxn) (from int, err error) { 315 schema := node.writeSchema 316 from = int(node.mustData().Length()) 317 for srcPos, attr := range bat.Attrs { 318 def := schema.ColDefs[schema.GetColIdx(attr)] 319 destVec := node.data.Vecs[def.Idx] 320 destVec.Extend(bat.Vecs[srcPos]) 321 } 322 return 323 } 324 325 func (node *memoryNode) GetRowByFilter( 326 ctx context.Context, 327 txn txnif.TxnReader, 328 filter *handle.Filter, 329 mp *mpool.MPool, 330 ) (blkID uint16, row uint32, err error) { 331 node.object.RLock() 332 defer node.object.RUnlock() 333 rows, err := node.GetRowsByKey(filter.Val) 334 if err != nil && !moerr.IsMoErrCode(err, moerr.ErrNotFound) { 335 return 336 } 337 338 waitFn := func(n *updates.AppendNode) { 339 txn := n.Txn 340 if txn != nil { 341 node.object.RUnlock() 342 txn.GetTxnState(true) 343 node.object.RLock() 344 } 345 } 346 if anyWaitable := node.object.appendMVCC.CollectUncommittedANodesPreparedBefore( 347 txn.GetStartTS(), 348 waitFn); anyWaitable { 349 rows, err = node.GetRowsByKey(filter.Val) 350 if err != nil { 351 return 352 } 353 } 354 355 for i := len(rows) - 1; i >= 0; i-- { 356 row = rows[i] 357 appendnode := node.object.appendMVCC.GetAppendNodeByRow(row) 358 needWait, waitTxn := appendnode.NeedWaitCommitting(txn.GetStartTS()) 359 if needWait { 360 node.object.RUnlock() 361 waitTxn.GetTxnState(true) 362 node.object.RLock() 363 } 364 if appendnode.IsAborted() || !appendnode.IsVisible(txn) { 365 continue 366 } 367 objMVCC := node.object.tryGetMVCC() 368 if objMVCC == nil { 369 return 370 } 371 var deleted bool 372 deleted, err = objMVCC.IsDeletedLocked(row, txn, 0) 373 if err != nil { 374 return 375 } 376 if !deleted { 377 return 378 } 379 } 380 return 0, 0, moerr.NewNotFoundNoCtx() 381 } 382 383 func (node *memoryNode) BatchDedup( 384 ctx context.Context, 385 txn txnif.TxnReader, 386 isCommitting bool, 387 keys containers.Vector, 388 keysZM index.ZM, 389 rowmask *roaring.Bitmap, 390 bf objectio.BloomFilter, 391 ) (err error) { 392 var dupRow uint32 393 node.object.RLock() 394 defer node.object.RUnlock() 395 _, err = node.doBatchDedup( 396 ctx, 397 keys, 398 keysZM, 399 node.checkConflictAndDupClosure(txn, isCommitting, &dupRow, rowmask), 400 bf) 401 402 // definitely no duplicate 403 if err == nil || !moerr.IsMoErrCode(err, moerr.OkExpectedDup) { 404 return 405 } 406 def := node.writeSchema.GetSingleSortKey() 407 v, isNull := node.GetValueByRow(node.writeSchema, int(dupRow), def.Idx) 408 entry := common.TypeStringValue(*keys.GetType(), v, isNull) 409 return moerr.NewDuplicateEntryNoCtx(entry, def.Name) 410 } 411 412 func (node *memoryNode) checkConflictAndDupClosure( 413 txn txnif.TxnReader, 414 isCommitting bool, 415 dupRow *uint32, 416 rowmask *roaring.Bitmap, 417 ) func(row uint32) error { 418 return func(row uint32) (err error) { 419 if rowmask != nil && rowmask.Contains(row) { 420 return nil 421 } 422 appendnode := node.object.appendMVCC.GetAppendNodeByRow(row) 423 var visible bool 424 if visible, err = node.checkConflictAandVisibility( 425 appendnode, 426 isCommitting, 427 txn); err != nil { 428 return 429 } 430 if appendnode.IsAborted() || !visible { 431 return nil 432 } 433 objMVCC := node.object.tryGetMVCC() 434 if objMVCC == nil { 435 *dupRow = row 436 return moerr.GetOkExpectedDup() 437 } 438 mvcc := objMVCC.TryGetDeleteChain(0) 439 if mvcc == nil { 440 *dupRow = row 441 return moerr.GetOkExpectedDup() 442 } 443 deleteNode := mvcc.GetDeleteNodeByRow(row) 444 if deleteNode == nil { 445 *dupRow = row 446 return moerr.GetOkExpectedDup() 447 } 448 449 if visible, err = node.checkConflictAandVisibility( 450 deleteNode, 451 isCommitting, 452 txn); err != nil { 453 return 454 } 455 if deleteNode.IsAborted() || !visible { 456 return moerr.GetOkExpectedDup() 457 } 458 return nil 459 } 460 } 461 462 func (node *memoryNode) checkConflictAandVisibility( 463 n txnif.BaseMVCCNode, 464 isCommitting bool, 465 txn txnif.TxnReader, 466 ) (visible bool, err error) { 467 // if isCommitting check all nodes commit before txn.CommitTS(PrepareTS) 468 // if not isCommitting check nodes commit before txn.StartTS 469 if isCommitting { 470 needWait := n.IsCommitting() 471 if needWait { 472 txn := n.GetTxn() 473 node.object.RUnlock() 474 txn.GetTxnState(true) 475 node.object.RLock() 476 } 477 } else { 478 needWait, txn := n.NeedWaitCommitting(txn.GetStartTS()) 479 if needWait { 480 node.object.RUnlock() 481 txn.GetTxnState(true) 482 node.object.RLock() 483 } 484 } 485 if err = n.CheckConflict(txn); err != nil { 486 return 487 } 488 if isCommitting { 489 visible = n.IsCommitted() 490 } else { 491 visible = n.IsVisible(txn) 492 } 493 return 494 } 495 496 func (node *memoryNode) CollectAppendInRange( 497 start, end types.TS, withAborted bool, mp *mpool.MPool, 498 ) (batWithVer *containers.BatchWithVersion, err error) { 499 node.object.RLock() 500 minRow, maxRow, commitTSVec, abortVec, abortedMap := 501 node.object.appendMVCC.CollectAppendLocked(start, end, mp) 502 batWithVer, err = node.GetDataWindowOnWriteSchema(minRow, maxRow, mp) 503 if err != nil { 504 node.object.RUnlock() 505 return nil, err 506 } 507 node.object.RUnlock() 508 509 batWithVer.Seqnums = append(batWithVer.Seqnums, objectio.SEQNUM_COMMITTS) 510 batWithVer.AddVector(catalog.AttrCommitTs, commitTSVec) 511 if withAborted { 512 batWithVer.Seqnums = append(batWithVer.Seqnums, objectio.SEQNUM_ABORT) 513 batWithVer.AddVector(catalog.AttrAborted, abortVec) 514 } else { 515 abortVec.Close() 516 batWithVer.Deletes = abortedMap 517 batWithVer.Compact() 518 } 519 520 return 521 } 522 523 // Note: With PinNode Context 524 func (node *memoryNode) resolveInMemoryColumnDatas( 525 ctx context.Context, 526 txn txnif.TxnReader, 527 readSchema *catalog.Schema, 528 colIdxes []int, 529 skipDeletes bool, 530 mp *mpool.MPool, 531 ) (view *containers.BlockView, err error) { 532 node.object.RLock() 533 defer node.object.RUnlock() 534 maxRow, visible, deSels, err := node.object.appendMVCC.GetVisibleRowLocked(ctx, txn) 535 if !visible || err != nil { 536 // blk.RUnlock() 537 return 538 } 539 data, err := node.GetDataWindow(readSchema, colIdxes, 0, maxRow, mp) 540 if err != nil { 541 return 542 } 543 view = containers.NewBlockView() 544 for i, colIdx := range colIdxes { 545 view.SetData(colIdx, data.Vecs[i]) 546 } 547 if skipDeletes { 548 return 549 } 550 551 err = node.object.fillInMemoryDeletesLocked(txn, 0, view.BaseView, node.object.RWMutex) 552 if err != nil { 553 return 554 } 555 if !deSels.IsEmpty() { 556 if view.DeleteMask != nil { 557 view.DeleteMask.Or(deSels) 558 } else { 559 view.DeleteMask = deSels 560 } 561 } 562 return 563 } 564 565 // Note: With PinNode Context 566 func (node *memoryNode) resolveInMemoryColumnData( 567 txn txnif.TxnReader, 568 readSchema *catalog.Schema, 569 col int, 570 skipDeletes bool, 571 mp *mpool.MPool, 572 ) (view *containers.ColumnView, err error) { 573 node.object.RLock() 574 defer node.object.RUnlock() 575 maxRow, visible, deSels, err := node.object.appendMVCC.GetVisibleRowLocked(context.TODO(), txn) 576 if !visible || err != nil { 577 return 578 } 579 580 view = containers.NewColumnView(col) 581 var data containers.Vector 582 if data, err = node.GetColumnDataWindow( 583 readSchema, 584 0, 585 maxRow, 586 col, 587 mp, 588 ); err != nil { 589 return 590 } 591 view.SetData(data) 592 if skipDeletes { 593 return 594 } 595 596 err = node.object.fillInMemoryDeletesLocked(txn, 0, view.BaseView, node.object.RWMutex) 597 if err != nil { 598 return 599 } 600 if deSels != nil && !deSels.IsEmpty() { 601 if view.DeleteMask != nil { 602 view.DeleteMask.Or(deSels) 603 } else { 604 view.DeleteMask = deSels 605 } 606 } 607 608 return 609 } 610 611 // With PinNode Context 612 func (node *memoryNode) getInMemoryValue( 613 txn txnif.TxnReader, 614 readSchema *catalog.Schema, 615 row, col int, 616 mp *mpool.MPool, 617 ) (v any, isNull bool, err error) { 618 node.object.RLock() 619 deleted := false 620 objMVCC := node.object.tryGetMVCC() 621 if objMVCC != nil { 622 mvcc := objMVCC.TryGetDeleteChain(0) 623 if mvcc != nil { 624 deleted, err = mvcc.IsDeletedLocked(uint32(row), txn) 625 } 626 } 627 node.object.RUnlock() 628 if err != nil { 629 return 630 } 631 if deleted { 632 err = moerr.NewNotFoundNoCtx() 633 return 634 } 635 view, err := node.resolveInMemoryColumnData(txn, readSchema, col, true, mp) 636 if err != nil { 637 return 638 } 639 defer view.Close() 640 v, isNull = view.GetValue(row) 641 return 642 } 643 644 func (node *memoryNode) allRowsCommittedBefore(ts types.TS) bool { 645 node.object.RLock() 646 defer node.object.RUnlock() 647 return node.object.appendMVCC.AllAppendsCommittedBefore(ts) 648 }