github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/logtail/handle.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 logtail 16 17 /* 18 19 an application on logtail mgr: build reponse to SyncLogTailRequest 20 21 More docs: 22 https://github.com/matrixorigin/docs/blob/main/tech-notes/dnservice/ref_logtail_impl.md 23 24 25 Main workflow: 26 27 +------------------+ 28 | CheckpointRunner | 29 +------------------+ 30 ^ | 31 | range | ckp & newRange 32 | v 33 +------------------+ newRange +----------------+ snapshot +--------------+ 34 user -> | HandleGetLogTail | ---------> | LogtailManager | ----------> | LogtailTable | 35 ^ +------------------+ +----------------+ +--------------+ 36 | | 37 | +------------------+ | 38 +---------- | RespBuilder | ------------------>+-------------------+ 39 return +------------------+ | 40 entries | visit 41 | 42 v 43 +-----------------------------------+ 44 | txnblock2 | 45 ... +-----------------------------------+ ... 46 | bornTs | ... txn100 | txn101 |.. | 47 +-----------------+---------+-------+ 48 | | 49 | | 50 | | 51 +-----------------+ +----+-------+ dirty blocks 52 | | | | 53 v v v v 54 +-------+ +-------+ +-------+ 55 | BLK-1 | | BLK-2 | | BLK-3 | 56 +---+---+ +---+---+ +---+---+ 57 | | | 58 v v v 59 [V1@t25,disk] [V1@t17,mem] [V1@t17,disk] 60 | | | 61 v v v 62 [V0@t12,mem] [V0@t10,mem] [V0@t10,disk] 63 | | 64 v v 65 [V0@t7,mem] [V0@t7,mem] 66 67 68 */ 69 70 import ( 71 "context" 72 "fmt" 73 "sort" 74 "strconv" 75 "strings" 76 "time" 77 78 "github.com/matrixorigin/matrixone/pkg/container/batch" 79 80 "github.com/matrixorigin/matrixone/pkg/perfcounter" 81 82 "github.com/matrixorigin/matrixone/pkg/objectio" 83 84 pkgcatalog "github.com/matrixorigin/matrixone/pkg/catalog" 85 "github.com/matrixorigin/matrixone/pkg/common/mpool" 86 "github.com/matrixorigin/matrixone/pkg/container/types" 87 "github.com/matrixorigin/matrixone/pkg/fileservice" 88 "github.com/matrixorigin/matrixone/pkg/logutil" 89 "github.com/matrixorigin/matrixone/pkg/pb/api" 90 "github.com/matrixorigin/matrixone/pkg/util/fault" 91 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 92 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" 93 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" 94 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 95 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 96 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/iface/data" 97 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/txn/txnimpl" 98 "go.uber.org/zap" 99 ) 100 101 const Size90M = 90 * 1024 * 1024 102 103 type CheckpointClient interface { 104 CollectCheckpointsInRange(ctx context.Context, start, end types.TS) (ckpLoc string, lastEnd types.TS, err error) 105 FlushTable(ctx context.Context, dbID, tableID uint64, ts types.TS) error 106 } 107 108 func DecideTableScope(tableID uint64) Scope { 109 var scope Scope 110 switch tableID { 111 case pkgcatalog.MO_DATABASE_ID: 112 scope = ScopeDatabases 113 case pkgcatalog.MO_TABLES_ID: 114 scope = ScopeTables 115 case pkgcatalog.MO_COLUMNS_ID: 116 scope = ScopeColumns 117 default: 118 scope = ScopeUserTables 119 } 120 return scope 121 } 122 123 func HandleSyncLogTailReq( 124 ctx context.Context, 125 ckpClient CheckpointClient, 126 mgr *Manager, 127 c *catalog.Catalog, 128 req api.SyncLogTailReq, 129 canRetry bool) (resp api.SyncLogTailResp, closeCB func(), err error) { 130 now := time.Now() 131 logutil.Debugf("[Logtail] begin handle %+v", req) 132 defer func() { 133 if elapsed := time.Since(now); elapsed > 5*time.Second { 134 logutil.Infof("[Logtail] long pull cost %v, %v: %+v, %v ", elapsed, canRetry, req, err) 135 } 136 logutil.Debugf("[Logtail] end handle %d entries[%q], err %v", len(resp.Commands), resp.CkpLocation, err) 137 }() 138 start := types.BuildTS(req.CnHave.PhysicalTime, req.CnHave.LogicalTime) 139 end := types.BuildTS(req.CnWant.PhysicalTime, req.CnWant.LogicalTime) 140 did, tid := req.Table.DbId, req.Table.TbId 141 dbEntry, err := c.GetDatabaseByID(did) 142 if err != nil { 143 return 144 } 145 tableEntry, err := dbEntry.GetTableEntryByID(tid) 146 if err != nil { 147 return 148 } 149 tableEntry.RLock() 150 createTS := tableEntry.GetCreatedAtLocked() 151 tableEntry.RUnlock() 152 if start.Less(&createTS) { 153 start = createTS 154 } 155 156 ckpLoc, checkpointed, err := ckpClient.CollectCheckpointsInRange(ctx, start, end) 157 if err != nil { 158 return 159 } 160 161 if checkpointed.GreaterEq(&end) { 162 return api.SyncLogTailResp{ 163 CkpLocation: ckpLoc, 164 }, nil, err 165 } else if ckpLoc != "" { 166 start = checkpointed.Next() 167 } 168 169 scope := DecideTableScope(tid) 170 171 var visitor RespBuilder 172 173 if scope == ScopeUserTables { 174 visitor = NewTableLogtailRespBuilder(ctx, ckpLoc, start, end, tableEntry) 175 } else { 176 visitor = NewCatalogLogtailRespBuilder(ctx, scope, ckpLoc, start, end) 177 } 178 closeCB = visitor.Close 179 180 operator := mgr.GetTableOperator(start, end, c, did, tid, scope, visitor) 181 if err := operator.Run(); err != nil { 182 return api.SyncLogTailResp{}, visitor.Close, err 183 } 184 resp, err = visitor.BuildResp() 185 186 if canRetry && scope == ScopeUserTables { // check simple conditions first 187 _, name, forceFlush := fault.TriggerFault("logtail_max_size") 188 if (forceFlush && name == tableEntry.GetLastestSchemaLocked().Name) || resp.ProtoSize() > Size90M { 189 _ = ckpClient.FlushTable(ctx, did, tid, end) 190 // try again after flushing 191 newResp, closeCB, err := HandleSyncLogTailReq(ctx, ckpClient, mgr, c, req, false) 192 logutil.Infof("[logtail] flush result: %d -> %d err: %v", resp.ProtoSize(), newResp.ProtoSize(), err) 193 return newResp, closeCB, err 194 } 195 } 196 return 197 } 198 199 type RespBuilder interface { 200 catalog.Processor 201 BuildResp() (api.SyncLogTailResp, error) 202 Close() 203 } 204 205 // CatalogLogtailRespBuilder knows how to make api-entry from db and table entry. 206 // impl catalog.Processor interface, driven by BoundTableOperator 207 type CatalogLogtailRespBuilder struct { 208 ctx context.Context 209 *catalog.LoopProcessor 210 scope Scope 211 start, end types.TS 212 checkpoint string 213 insBatch *containers.Batch 214 delBatch *containers.Batch 215 specailDeleteBatch *containers.Batch 216 } 217 218 func NewCatalogLogtailRespBuilder(ctx context.Context, scope Scope, ckp string, start, end types.TS) *CatalogLogtailRespBuilder { 219 b := &CatalogLogtailRespBuilder{ 220 ctx: ctx, 221 LoopProcessor: new(catalog.LoopProcessor), 222 scope: scope, 223 start: start, 224 end: end, 225 checkpoint: ckp, 226 } 227 switch scope { 228 case ScopeDatabases: 229 b.insBatch = makeRespBatchFromSchema(catalog.SystemDBSchema, common.LogtailAllocator) 230 b.delBatch = makeRespBatchFromSchema(DBDelSchema, common.LogtailAllocator) 231 b.specailDeleteBatch = makeRespBatchFromSchema(DBSpecialDeleteSchema, common.LogtailAllocator) 232 case ScopeTables: 233 b.insBatch = makeRespBatchFromSchema(catalog.SystemTableSchema, common.LogtailAllocator) 234 b.delBatch = makeRespBatchFromSchema(TblDelSchema, common.LogtailAllocator) 235 b.specailDeleteBatch = makeRespBatchFromSchema(TBLSpecialDeleteSchema, common.LogtailAllocator) 236 case ScopeColumns: 237 b.insBatch = makeRespBatchFromSchema(catalog.SystemColumnSchema, common.LogtailAllocator) 238 b.delBatch = makeRespBatchFromSchema(ColumnDelSchema, common.LogtailAllocator) 239 } 240 b.DatabaseFn = b.VisitDB 241 b.TableFn = b.VisitTbl 242 243 return b 244 } 245 246 func (b *CatalogLogtailRespBuilder) Close() { 247 if b.insBatch != nil { 248 b.insBatch.Close() 249 b.insBatch = nil 250 } 251 if b.delBatch != nil { 252 b.delBatch.Close() 253 b.delBatch = nil 254 } 255 } 256 257 // VisitDB = catalog.Processor.OnDatabase 258 func (b *CatalogLogtailRespBuilder) VisitDB(entry *catalog.DBEntry) error { 259 entry.RLock() 260 if shouldIgnoreDBInLogtail(entry.ID) { 261 entry.RUnlock() 262 return nil 263 } 264 mvccNodes := entry.ClonePreparedInRangeLocked(b.start, b.end) 265 entry.RUnlock() 266 for _, node := range mvccNodes { 267 if node.IsAborted() { 268 continue 269 } 270 dbNode := node 271 if dbNode.HasDropCommitted() { 272 // delScehma is empty, it will just fill rowid / commit ts 273 catalogEntry2Batch(b.delBatch, entry, dbNode, DBDelSchema, txnimpl.FillDBRow, objectio.HackU64ToRowid(entry.GetID()), dbNode.GetEnd()) 274 catalogEntry2Batch(b.specailDeleteBatch, entry, node, DBSpecialDeleteSchema, txnimpl.FillDBRow, objectio.HackU64ToRowid(entry.GetID()), node.GetEnd()) 275 } else { 276 catalogEntry2Batch(b.insBatch, entry, dbNode, catalog.SystemDBSchema, txnimpl.FillDBRow, objectio.HackU64ToRowid(entry.GetID()), dbNode.GetEnd()) 277 } 278 } 279 return nil 280 } 281 282 // VisitTbl = catalog.Processor.OnTable 283 func (b *CatalogLogtailRespBuilder) VisitTbl(entry *catalog.TableEntry) error { 284 entry.RLock() 285 if shouldIgnoreTblInLogtail(entry.ID) { 286 entry.RUnlock() 287 return nil 288 } 289 mvccNodes := entry.ClonePreparedInRangeLocked(b.start, b.end) 290 entry.RUnlock() 291 for _, node := range mvccNodes { 292 if node.IsAborted() { 293 continue 294 } 295 if b.scope == ScopeColumns { 296 var dstBatch *containers.Batch 297 if !node.HasDropCommitted() { 298 dstBatch = b.insBatch 299 // fill unique syscol fields if inserting 300 for _, syscol := range catalog.SystemColumnSchema.ColDefs { 301 txnimpl.FillColumnRow(entry, node, syscol.Name, b.insBatch.GetVectorByName(syscol.Name)) 302 } 303 // send dropped column del 304 for _, name := range node.BaseNode.Schema.Extra.DroppedAttrs { 305 b.delBatch.GetVectorByName(catalog.AttrRowID).Append(objectio.HackBytes2Rowid([]byte(fmt.Sprintf("%d-%s", entry.GetID(), name))), false) 306 b.delBatch.GetVectorByName(catalog.AttrCommitTs).Append(node.GetEnd(), false) 307 b.delBatch.GetVectorByName(pkgcatalog.SystemColAttr_UniqName).Append([]byte(fmt.Sprintf("%d-%s", entry.GetID(), name)), false) 308 } 309 } else { 310 dstBatch = b.delBatch 311 } 312 313 // fill common syscol fields for every user column 314 rowidVec := dstBatch.GetVectorByName(catalog.AttrRowID) 315 commitVec := dstBatch.GetVectorByName(catalog.AttrCommitTs) 316 tableID := entry.GetID() 317 commitTs := node.GetEnd() 318 for _, usercol := range node.BaseNode.Schema.ColDefs { 319 rowidVec.Append(objectio.HackBytes2Rowid([]byte(fmt.Sprintf("%d-%s", tableID, usercol.Name))), false) 320 commitVec.Append(commitTs, false) 321 } 322 } else { 323 if node.HasDropCommitted() { 324 catalogEntry2Batch(b.delBatch, entry, node, TblDelSchema, txnimpl.FillTableRow, objectio.HackU64ToRowid(entry.GetID()), node.GetEnd()) 325 catalogEntry2Batch(b.specailDeleteBatch, entry, node, TBLSpecialDeleteSchema, txnimpl.FillTableRow, objectio.HackU64ToRowid(entry.GetID()), node.GetEnd()) 326 } else { 327 catalogEntry2Batch(b.insBatch, entry, node, catalog.SystemTableSchema, txnimpl.FillTableRow, objectio.HackU64ToRowid(entry.GetID()), node.GetEnd()) 328 } 329 } 330 } 331 return nil 332 } 333 334 func (b *CatalogLogtailRespBuilder) BuildResp() (api.SyncLogTailResp, error) { 335 entries := make([]*api.Entry, 0) 336 var tblID uint64 337 var tableName string 338 switch b.scope { 339 case ScopeDatabases: 340 tblID = pkgcatalog.MO_DATABASE_ID 341 tableName = pkgcatalog.MO_DATABASE 342 case ScopeTables: 343 tblID = pkgcatalog.MO_TABLES_ID 344 tableName = pkgcatalog.MO_TABLES 345 case ScopeColumns: 346 tblID = pkgcatalog.MO_COLUMNS_ID 347 tableName = pkgcatalog.MO_COLUMNS 348 } 349 350 if b.insBatch.Length() > 0 { 351 bat, err := containersBatchToProtoBatch(b.insBatch) 352 logutil.Debugf("[logtail] catalog insert to %d-%s, %s", tblID, tableName, 353 DebugBatchToString("catalog", b.insBatch, true, zap.DebugLevel)) 354 if err != nil { 355 return api.SyncLogTailResp{}, err 356 } 357 insEntry := &api.Entry{ 358 EntryType: api.Entry_Insert, 359 TableId: tblID, 360 TableName: tableName, 361 DatabaseId: pkgcatalog.MO_CATALOG_ID, 362 DatabaseName: pkgcatalog.MO_CATALOG, 363 Bat: bat, 364 } 365 entries = append(entries, insEntry) 366 perfcounter.Update(b.ctx, func(counter *perfcounter.CounterSet) { 367 counter.TAE.LogTail.Entries.Add(int64(b.insBatch.Length())) 368 counter.TAE.LogTail.InsertEntries.Add(int64(b.insBatch.Length())) 369 }) 370 } 371 if b.delBatch.Length() > 0 { 372 bat, err := containersBatchToProtoBatch(b.delBatch) 373 logutil.Debugf("[logtail] catalog delete from %d-%s, %s", tblID, tableName, 374 DebugBatchToString("catalog", b.delBatch, false, zap.DebugLevel)) 375 if err != nil { 376 return api.SyncLogTailResp{}, err 377 } 378 delEntry := &api.Entry{ 379 EntryType: api.Entry_Delete, 380 TableId: tblID, 381 TableName: tableName, 382 DatabaseId: pkgcatalog.MO_CATALOG_ID, 383 DatabaseName: pkgcatalog.MO_CATALOG, 384 Bat: bat, 385 } 386 entries = append(entries, delEntry) 387 perfcounter.Update(b.ctx, func(counter *perfcounter.CounterSet) { 388 counter.TAE.LogTail.Entries.Add(int64(b.delBatch.Length())) 389 counter.TAE.LogTail.DeleteEntries.Add(int64(b.delBatch.Length())) 390 }) 391 } 392 if b.specailDeleteBatch != nil && b.specailDeleteBatch.Length() > 0 { 393 bat, err := containersBatchToProtoBatch(b.specailDeleteBatch) 394 if err != nil { 395 return api.SyncLogTailResp{}, err 396 } 397 delEntry := &api.Entry{ 398 EntryType: api.Entry_SpecialDelete, 399 TableId: tblID, 400 TableName: tableName, 401 DatabaseId: pkgcatalog.MO_CATALOG_ID, 402 DatabaseName: pkgcatalog.MO_CATALOG, 403 Bat: bat, 404 } 405 entries = append(entries, delEntry) 406 perfcounter.Update(b.ctx, func(counter *perfcounter.CounterSet) { 407 counter.TAE.LogTail.Entries.Add(int64(b.delBatch.Length())) 408 counter.TAE.LogTail.DeleteEntries.Add(int64(b.delBatch.Length())) 409 }) 410 } 411 return api.SyncLogTailResp{ 412 CkpLocation: b.checkpoint, 413 Commands: entries, 414 }, nil 415 } 416 417 // this is used to collect ONE ROW of db or table change 418 func catalogEntry2Batch[ 419 T *catalog.DBEntry | *catalog.TableEntry, 420 N *catalog.MVCCNode[*catalog.EmptyMVCCNode] | *catalog.MVCCNode[*catalog.TableMVCCNode]]( 421 dstBatch *containers.Batch, 422 e T, 423 node N, 424 schema *catalog.Schema, 425 fillDataRow func(e T, node N, attr string, col containers.Vector), 426 rowid types.Rowid, 427 commitTs types.TS, 428 ) { 429 for _, col := range schema.ColDefs { 430 fillDataRow(e, node, col.Name, dstBatch.GetVectorByName(col.Name)) 431 } 432 dstBatch.GetVectorByName(catalog.AttrRowID).Append(rowid, false) 433 dstBatch.GetVectorByName(catalog.AttrCommitTs).Append(commitTs, false) 434 } 435 436 // CatalogLogtailRespBuilder knows how to make api-entry from block entry. 437 // impl catalog.Processor interface, driven by BoundTableOperator 438 type TableLogtailRespBuilder struct { 439 ctx context.Context 440 *catalog.LoopProcessor 441 start, end types.TS 442 did, tid uint64 443 dname, tname string 444 checkpoint string 445 blkMetaInsBatch *containers.Batch 446 blkMetaDelBatch *containers.Batch 447 objectMetaBatch *containers.Batch 448 dataInsBatches map[uint32]*containers.Batch // schema version -> data batch 449 dataDelBatch *containers.Batch 450 } 451 452 func NewTableLogtailRespBuilder(ctx context.Context, ckp string, start, end types.TS, tbl *catalog.TableEntry) *TableLogtailRespBuilder { 453 b := &TableLogtailRespBuilder{ 454 ctx: ctx, 455 LoopProcessor: new(catalog.LoopProcessor), 456 start: start, 457 end: end, 458 checkpoint: ckp, 459 } 460 b.ObjectFn = b.VisitObj 461 b.TombstoneFn = b.visitDelete 462 463 b.did = tbl.GetDB().GetID() 464 b.tid = tbl.ID 465 b.dname = tbl.GetDB().GetName() 466 b.tname = tbl.GetLastestSchemaLocked().Name 467 468 b.dataInsBatches = make(map[uint32]*containers.Batch) 469 b.blkMetaInsBatch = makeRespBatchFromSchema(BlkMetaSchema, common.LogtailAllocator) 470 b.blkMetaDelBatch = makeRespBatchFromSchema(DelSchema, common.LogtailAllocator) 471 b.objectMetaBatch = makeRespBatchFromSchema(ObjectInfoSchema, common.LogtailAllocator) 472 return b 473 } 474 475 func (b *TableLogtailRespBuilder) Close() { 476 for _, vec := range b.dataInsBatches { 477 if vec != nil { 478 vec.Close() 479 } 480 } 481 b.dataInsBatches = nil 482 if b.dataDelBatch != nil { 483 b.dataDelBatch.Close() 484 b.dataDelBatch = nil 485 } 486 if b.blkMetaInsBatch != nil { 487 b.blkMetaInsBatch.Close() 488 b.blkMetaInsBatch = nil 489 } 490 if b.blkMetaDelBatch != nil { 491 b.blkMetaDelBatch.Close() 492 b.blkMetaDelBatch = nil 493 } 494 } 495 496 func (b *TableLogtailRespBuilder) VisitObj(e *catalog.ObjectEntry) error { 497 skip, err := b.visitObjMeta(e) 498 if err != nil { 499 return err 500 } 501 if skip { 502 return nil 503 } else { 504 return b.visitObjData(e) 505 } 506 } 507 func (b *TableLogtailRespBuilder) visitObjMeta(e *catalog.ObjectEntry) (bool, error) { 508 mvccNodes := e.ClonePreparedInRange(b.start, b.end) 509 if len(mvccNodes) == 0 { 510 return false, nil 511 } 512 513 for _, node := range mvccNodes { 514 if e.IsAppendable() && node.BaseNode.IsEmpty() { 515 continue 516 } 517 visitObject(b.objectMetaBatch, e, node, false, types.TS{}) 518 } 519 return b.skipObjectData(e, mvccNodes[len(mvccNodes)-1]), nil 520 } 521 func (b *TableLogtailRespBuilder) skipObjectData(e *catalog.ObjectEntry, lastMVCCNode *catalog.MVCCNode[*catalog.ObjectMVCCNode]) bool { 522 if e.IsAppendable() { 523 // appendable block has been flushed, no need to collect data 524 return !lastMVCCNode.BaseNode.IsEmpty() 525 } else { 526 return true 527 } 528 } 529 func (b *TableLogtailRespBuilder) visitObjData(e *catalog.ObjectEntry) error { 530 data := e.GetObjectData() 531 insBatch, err := data.CollectAppendInRange(b.start, b.end, false, common.LogtailAllocator) 532 if err != nil { 533 return err 534 } 535 if insBatch != nil && insBatch.Length() > 0 { 536 dest, ok := b.dataInsBatches[insBatch.Version] 537 if !ok { 538 // create new dest batch 539 dest = DataChangeToLogtailBatch(insBatch) 540 b.dataInsBatches[insBatch.Version] = dest 541 } else { 542 dest.Extend(insBatch.Batch) 543 // insBatch is freed, don't use anymore 544 } 545 } 546 return nil 547 } 548 func visitObject(batch *containers.Batch, entry *catalog.ObjectEntry, node *catalog.MVCCNode[*catalog.ObjectMVCCNode], push bool, committs types.TS) { 549 batch.GetVectorByName(catalog.AttrRowID).Append(objectio.HackObjid2Rowid(&entry.ID), false) 550 if push { 551 batch.GetVectorByName(catalog.AttrCommitTs).Append(committs, false) 552 } else { 553 batch.GetVectorByName(catalog.AttrCommitTs).Append(node.TxnMVCCNode.End, false) 554 } 555 node.BaseNode.AppendTuple(&entry.ID, batch) 556 if push { 557 node.TxnMVCCNode.AppendTupleWithCommitTS(batch, committs) 558 } else { 559 node.TxnMVCCNode.AppendTuple(batch) 560 } 561 if push { 562 node.EntryMVCCNode.AppendTupleWithCommitTS(batch, committs) 563 } else { 564 node.EntryMVCCNode.AppendTuple(batch) 565 } 566 batch.GetVectorByName(SnapshotAttr_DBID).Append(entry.GetTable().GetDB().ID, false) 567 batch.GetVectorByName(SnapshotAttr_TID).Append(entry.GetTable().ID, false) 568 batch.GetVectorByName(ObjectAttr_State).Append(entry.IsAppendable(), false) 569 sorted := false 570 if entry.GetTable().GetLastestSchemaLocked().HasSortKey() && !entry.IsAppendable() { 571 sorted = true 572 } 573 batch.GetVectorByName(ObjectAttr_Sorted).Append(sorted, false) 574 } 575 576 func (b *TableLogtailRespBuilder) visitDelete(e data.Tombstone) error { 577 deletes, _, _, err := e.VisitDeletes(b.ctx, b.start, b.end, b.blkMetaInsBatch, nil, false) 578 if err != nil { 579 return err 580 } 581 if deletes != nil && deletes.Length() != 0 { 582 if b.dataDelBatch == nil { 583 b.dataDelBatch = deletes 584 } else { 585 b.dataDelBatch.Extend(deletes) 586 deletes.Close() 587 } 588 } 589 return nil 590 } 591 592 type TableRespKind int 593 594 const ( 595 TableRespKind_Data TableRespKind = iota 596 TableRespKind_Blk 597 TableRespKind_Obj 598 ) 599 600 func (b *TableLogtailRespBuilder) BuildResp() (api.SyncLogTailResp, error) { 601 entries := make([]*api.Entry, 0) 602 tryAppendEntry := func(typ api.Entry_EntryType, kind TableRespKind, batch *containers.Batch, version uint32) error { 603 if batch == nil || batch.Length() == 0 { 604 return nil 605 } 606 bat, err := containersBatchToProtoBatch(batch) 607 if err != nil { 608 return err 609 } 610 611 tableName := "" 612 switch kind { 613 case TableRespKind_Data: 614 tableName = b.tname 615 logutil.Debugf("[logtail] table data [%v] %d-%s-%d: %s", typ, b.tid, b.tname, version, 616 DebugBatchToString("data", batch, false, zap.InfoLevel)) 617 case TableRespKind_Blk: 618 tableName = fmt.Sprintf("_%d_meta", b.tid) 619 logutil.Debugf("[logtail] table meta [%v] %d-%s: %s", typ, b.tid, b.tname, 620 DebugBatchToString("blkmeta", batch, false, zap.InfoLevel)) 621 case TableRespKind_Obj: 622 tableName = fmt.Sprintf("_%d_obj", b.tid) 623 logutil.Debugf("[logtail] table meta [%v] %d-%s: %s", typ, b.tid, b.tname, 624 DebugBatchToString("object", batch, false, zap.InfoLevel)) 625 } 626 627 entry := &api.Entry{ 628 EntryType: typ, 629 TableId: b.tid, 630 TableName: tableName, 631 DatabaseId: b.did, 632 DatabaseName: b.dname, 633 Bat: bat, 634 } 635 entries = append(entries, entry) 636 return nil 637 } 638 639 empty := api.SyncLogTailResp{} 640 if err := tryAppendEntry(api.Entry_Insert, TableRespKind_Blk, b.blkMetaInsBatch, 0); err != nil { 641 return empty, err 642 } 643 if err := tryAppendEntry(api.Entry_Delete, TableRespKind_Blk, b.blkMetaDelBatch, 0); err != nil { 644 return empty, err 645 } 646 if err := tryAppendEntry(api.Entry_Insert, TableRespKind_Obj, b.objectMetaBatch, 0); err != nil { 647 return empty, err 648 } 649 keys := make([]uint32, 0, len(b.dataInsBatches)) 650 for k := range b.dataInsBatches { 651 keys = append(keys, k) 652 } 653 sort.Slice(keys, func(i, j int) bool { return keys[i] < keys[j] }) 654 for _, k := range keys { 655 if err := tryAppendEntry(api.Entry_Insert, TableRespKind_Data, b.dataInsBatches[k], k); err != nil { 656 return empty, err 657 } 658 } 659 if err := tryAppendEntry(api.Entry_Delete, TableRespKind_Data, b.dataDelBatch, 0); err != nil { 660 return empty, err 661 } 662 663 return api.SyncLogTailResp{ 664 CkpLocation: b.checkpoint, 665 Commands: entries, 666 }, nil 667 } 668 func GetMetaIdxesByVersion(ver uint32) []uint16 { 669 meteIdxSchema := checkpointDataReferVersions[ver][MetaIDX] 670 idxes := make([]uint16, len(meteIdxSchema.attrs)) 671 for attr := range meteIdxSchema.attrs { 672 idxes[attr] = uint16(attr) 673 } 674 return idxes 675 } 676 func LoadCheckpointEntries( 677 ctx context.Context, 678 metLoc string, 679 tableID uint64, 680 tableName string, 681 dbID uint64, 682 dbName string, 683 mp *mpool.MPool, 684 fs fileservice.FileService) ([]*api.Entry, []func(), error) { 685 if metLoc == "" { 686 return nil, nil, nil 687 } 688 v2.LogtailLoadCheckpointCounter.Inc() 689 now := time.Now() 690 defer func() { 691 v2.LogTailLoadCheckpointDurationHistogram.Observe(time.Since(now).Seconds()) 692 }() 693 locationsAndVersions := strings.Split(metLoc, ";") 694 695 datas := make([]*CNCheckpointData, len(locationsAndVersions)/2) 696 697 readers := make([]*blockio.BlockReader, len(locationsAndVersions)/2) 698 objectLocations := make([]objectio.Location, len(locationsAndVersions)/2) 699 versions := make([]uint32, len(locationsAndVersions)/2) 700 locations := make([]objectio.Location, len(locationsAndVersions)/2) 701 for i := 0; i < len(locationsAndVersions); i += 2 { 702 key := locationsAndVersions[i] 703 version, err := strconv.ParseUint(locationsAndVersions[i+1], 10, 32) 704 if err != nil { 705 return nil, nil, err 706 } 707 location, err := blockio.EncodeLocationFromString(key) 708 if err != nil { 709 return nil, nil, err 710 } 711 locations[i/2] = location 712 reader, err := blockio.NewObjectReader(fs, location) 713 if err != nil { 714 return nil, nil, err 715 } 716 readers[i/2] = reader 717 err = blockio.PrefetchMeta(fs, location) 718 if err != nil { 719 return nil, nil, err 720 } 721 objectLocations[i/2] = location 722 versions[i/2] = uint32(version) 723 } 724 725 for i := range objectLocations { 726 data := NewCNCheckpointData() 727 meteIdxSchema := checkpointDataReferVersions[versions[i]][MetaIDX] 728 idxes := make([]uint16, len(meteIdxSchema.attrs)) 729 for attr := range meteIdxSchema.attrs { 730 idxes[attr] = uint16(attr) 731 } 732 err := data.PrefetchMetaIdx(ctx, versions[i], idxes, objectLocations[i], fs) 733 if err != nil { 734 return nil, nil, err 735 } 736 datas[i] = data 737 } 738 739 for i := range datas { 740 err := datas[i].InitMetaIdx(ctx, versions[i], readers[i], locations[i], mp) 741 if err != nil { 742 return nil, nil, err 743 } 744 } 745 746 for i := range datas { 747 err := datas[i].PrefetchMetaFrom(ctx, versions[i], locations[i], fs, tableID) 748 if err != nil { 749 return nil, nil, err 750 } 751 } 752 753 for i := range datas { 754 err := datas[i].PrefetchFrom(ctx, versions[i], fs, locations[i], tableID) 755 if err != nil { 756 return nil, nil, err 757 } 758 } 759 760 closeCBs := make([]func(), 0) 761 bats := make([][]*batch.Batch, len(locationsAndVersions)/2) 762 var err error 763 for i, data := range datas { 764 var bat []*batch.Batch 765 bat, err = data.ReadFromData(ctx, tableID, locations[i], readers[i], versions[i], mp) 766 closeCBs = append(closeCBs, data.GetCloseCB(versions[i], mp)) 767 if err != nil { 768 for j := range closeCBs { 769 if closeCBs[j] != nil { 770 closeCBs[j]() 771 } 772 } 773 return nil, nil, err 774 } 775 bats[i] = bat 776 } 777 778 entries := make([]*api.Entry, 0) 779 for i := range objectLocations { 780 data := datas[i] 781 ins, del, cnIns, objInfo, err := data.GetTableDataFromBats(tableID, bats[i]) 782 if err != nil { 783 for j := range closeCBs { 784 if closeCBs[j] != nil { 785 closeCBs[j]() 786 } 787 } 788 return nil, nil, err 789 } 790 if tableName != pkgcatalog.MO_DATABASE && 791 tableName != pkgcatalog.MO_COLUMNS && 792 tableName != pkgcatalog.MO_TABLES { 793 tableName = fmt.Sprintf("_%d_meta", tableID) 794 } 795 if ins != nil { 796 entry := &api.Entry{ 797 EntryType: api.Entry_Insert, 798 TableId: tableID, 799 TableName: tableName, 800 DatabaseId: dbID, 801 DatabaseName: dbName, 802 Bat: ins, 803 } 804 entries = append(entries, entry) 805 } 806 if cnIns != nil { 807 entry := &api.Entry{ 808 EntryType: api.Entry_Insert, 809 TableId: tableID, 810 TableName: tableName, 811 DatabaseId: dbID, 812 DatabaseName: dbName, 813 Bat: cnIns, 814 } 815 entries = append(entries, entry) 816 } 817 if del != nil { 818 entry := &api.Entry{ 819 EntryType: api.Entry_Delete, 820 TableId: tableID, 821 TableName: tableName, 822 DatabaseId: dbID, 823 DatabaseName: dbName, 824 Bat: del, 825 } 826 entries = append(entries, entry) 827 } 828 if objInfo != nil { 829 entry := &api.Entry{ 830 EntryType: api.Entry_Insert, 831 TableId: tableID, 832 TableName: fmt.Sprintf("_%d_obj", tableID), 833 DatabaseId: dbID, 834 DatabaseName: dbName, 835 Bat: objInfo, 836 } 837 entries = append(entries, entry) 838 } 839 } 840 return entries, closeCBs, nil 841 }