github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/db/checkpoint/runner.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 checkpoint 16 17 import ( 18 "context" 19 "strings" 20 "sync" 21 "sync/atomic" 22 "time" 23 24 "github.com/google/uuid" 25 "github.com/matrixorigin/matrixone/pkg/common/moerr" 26 "github.com/matrixorigin/matrixone/pkg/common/stopper" 27 "github.com/matrixorigin/matrixone/pkg/container/types" 28 "github.com/matrixorigin/matrixone/pkg/logutil" 29 "github.com/matrixorigin/matrixone/pkg/objectio" 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/dataio/blockio" 33 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" 34 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" 35 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks" 36 w "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks/worker" 37 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal" 38 "github.com/tidwall/btree" 39 ) 40 41 type timeBasedPolicy struct { 42 interval time.Duration 43 } 44 45 func (p *timeBasedPolicy) Check(last types.TS) bool { 46 physical := last.Physical() 47 return physical <= time.Now().UTC().UnixNano()-p.interval.Nanoseconds() 48 } 49 50 type countBasedPolicy struct { 51 minCount int 52 current int 53 } 54 55 func (p *countBasedPolicy) Check() bool { 56 return p.current >= p.minCount 57 } 58 59 func (p *countBasedPolicy) Add(cnt int) { 60 p.current++ 61 } 62 63 func (p *countBasedPolicy) Reset() { 64 p.current = 0 65 } 66 67 type globalCheckpointContext struct { 68 force bool 69 end types.TS 70 interval time.Duration 71 } 72 73 // Q: What does runner do? 74 // A: A checkpoint runner organizes and manages all checkpoint-related behaviors. It roughly 75 // does the following things: 76 // - Manage the life cycle of all checkpoints and provide some query interfaces. 77 // - A cron job periodically collects and analyzes dirty blocks, and flushes eligibl dirty 78 // blocks to the remote storage 79 // - The cron job peridically test whether a new checkpoint can be created. If it is not 80 // satisfied, it will wait for next trigger. Otherwise, it will start the process of 81 // creating a checkpoint. 82 83 // Q: How to collect dirty blocks? 84 // A: There is a logtail manager maintains all transaction information that occurred over a 85 // period of time. When a checkpoint is generated, we clean up the data before the 86 // checkpoint timestamp in logtail. 87 // 88 // |----to prune----| Time 89 // -----+----------------+-------------------------------------+-----------> 90 // t1 checkpoint-t10 t100 91 // 92 // For each transaction, it maintains a dirty block list. 93 // 94 // [t1]: TB1-[1] 95 // [t2]: TB2-[2] 96 // [t3]: TB1-[1],TB2-[3] 97 // [t4]: [] 98 // [t5]: TB2[3,4] 99 // ..... 100 // ..... 101 // When collecting the dirty blocks in [t1, t5], it will get 2 block list, which is represented 102 // with `common.Tree` 103 // [t1,t5] - - - - - - - - - <DirtyTreeEntry> 104 // / \ 105 // [TB1] [TB2] 106 // | | 107 // [1] [2,3,4] - - - - - - - leaf nodes are all dirty blocks 108 // We store the dirty tree entries into the internal storage. Over time, we'll see something like 109 // this inside the storage: 110 // - Entry[t1, t5] 111 // - Entry[t6, t12] 112 // - Entry[t13,t29] 113 // - Entry[t30,t47] 114 // ..... 115 // ..... 116 // When collecting the dirty blocks in [t1, t20], it will get 3 dirty trees from [t1,t5],[t6,t12], 117 // [t13,t29] and merge the three trees into a tree with all the leaf nodes being dirty blocks. 118 // 119 // In order to reduce the workload of scan, we have always been incremental scan. And also we will 120 // continue to clean up the entries in the storage. 121 122 // Q: How to test whether a block need to be flushed? 123 // A: It is an open question. There are a lot of options, just chose a simple strategy for now. 124 // Must: 125 // - The born transaction of the block was committed 126 // - No uncommitted transaction on the block 127 // Factors: 128 // - Max rows reached 129 // - Delete ratio 130 // - Max flush timeout 131 132 // Q: How to do incremental checkpoint? 133 // A: 1. Decide a checkpoint timestamp 134 // 2. Wait all transactions before timestamp were committed 135 // 3. Wait all dirty blocks before the timestamp were flushed 136 // 4. Prepare checkpoint data 137 // 5. Persist the checkpoint data 138 // 6. Persist the checkpoint meta data 139 // 7. Notify checkpoint events to all the observers 140 // 8. Schedule to remove stale checkpoint meta objects 141 142 // Q: How to boot from the checkpoints? 143 // A: When a meta version is created, it contains all information of the previouse version. So we always 144 // 145 // delete the stale versions when a new version is created. Over time, the number of objects under 146 // `ckp/` is small. 147 // 1. List all meta objects under `ckp/`. Get the latest meta object and read all checkpoint informations 148 // from the meta object. 149 // 2. Apply the latest global checkpoint 150 // 3. Apply the incremental checkpoint start from the version right after the global checkpoint to the 151 // latest version. 152 type runner struct { 153 options struct { 154 // checkpoint scanner interval duration 155 collectInterval time.Duration 156 157 // maximum dirty block flush interval duration 158 maxFlushInterval time.Duration 159 160 // minimum incremental checkpoint interval duration 161 minIncrementalInterval time.Duration 162 163 // minimum global checkpoint interval duration 164 globalMinCount int 165 forceUpdateGlobalInterval bool 166 globalVersionInterval time.Duration 167 168 // minimum count of uncheckpointed transactions allowed before the next checkpoint 169 minCount int 170 171 forceFlushTimeout time.Duration 172 forceFlushCheckInterval time.Duration 173 174 dirtyEntryQueueSize int 175 waitQueueSize int 176 checkpointQueueSize int 177 } 178 179 // logtail sourcer 180 source logtail.Collector 181 catalog *catalog.Catalog 182 scheduler tasks.TaskScheduler 183 fs *objectio.ObjectFS 184 observers *observers 185 wal wal.Driver 186 disabled atomic.Bool 187 188 stopper *stopper.Stopper 189 190 // memory storage of the checkpoint entries 191 storage struct { 192 sync.RWMutex 193 entries *btree.BTreeG[*CheckpointEntry] 194 globals *btree.BTreeG[*CheckpointEntry] 195 } 196 197 gcTS atomic.Value 198 199 // checkpoint policy 200 incrementalPolicy *timeBasedPolicy 201 globalPolicy *countBasedPolicy 202 203 dirtyEntryQueue sm.Queue 204 waitQueue sm.Queue 205 incrementalCheckpointQueue sm.Queue 206 globalCheckpointQueue sm.Queue 207 postCheckpointQueue sm.Queue 208 gcCheckpointQueue sm.Queue 209 210 onceStart sync.Once 211 onceStop sync.Once 212 } 213 214 func NewRunner( 215 fs *objectio.ObjectFS, 216 catalog *catalog.Catalog, 217 scheduler tasks.TaskScheduler, 218 source logtail.Collector, 219 wal wal.Driver, 220 opts ...Option) *runner { 221 r := &runner{ 222 catalog: catalog, 223 scheduler: scheduler, 224 source: source, 225 fs: fs, 226 observers: new(observers), 227 wal: wal, 228 } 229 r.storage.entries = btree.NewBTreeGOptions(func(a, b *CheckpointEntry) bool { 230 return a.end.Less(b.end) 231 }, btree.Options{ 232 NoLocks: true, 233 }) 234 r.storage.globals = btree.NewBTreeGOptions(func(a, b *CheckpointEntry) bool { 235 return a.end.Less(b.end) 236 }, btree.Options{ 237 NoLocks: true, 238 }) 239 for _, opt := range opts { 240 opt(r) 241 } 242 r.fillDefaults() 243 244 r.incrementalPolicy = &timeBasedPolicy{interval: r.options.minIncrementalInterval} 245 r.globalPolicy = &countBasedPolicy{minCount: r.options.globalMinCount} 246 r.stopper = stopper.NewStopper("CheckpointRunner") 247 r.dirtyEntryQueue = sm.NewSafeQueue(r.options.dirtyEntryQueueSize, 100, r.onDirtyEntries) 248 r.waitQueue = sm.NewSafeQueue(r.options.waitQueueSize, 100, r.onWaitWaitableItems) 249 r.incrementalCheckpointQueue = sm.NewSafeQueue(r.options.checkpointQueueSize, 100, r.onIncrementalCheckpointEntries) 250 r.globalCheckpointQueue = sm.NewSafeQueue(r.options.checkpointQueueSize, 100, r.onGlobalCheckpointEntries) 251 r.gcCheckpointQueue = sm.NewSafeQueue(100, 100, r.onGCCheckpointEntries) 252 r.postCheckpointQueue = sm.NewSafeQueue(1000, 1, r.onPostCheckpointEntries) 253 return r 254 } 255 256 // Only used in UT 257 func (r *runner) DebugUpdateOptions(opts ...Option) { 258 for _, opt := range opts { 259 opt(r) 260 } 261 } 262 263 func (r *runner) onGlobalCheckpointEntries(items ...any) { 264 for _, item := range items { 265 ctx := item.(*globalCheckpointContext) 266 doCheckpoint := false 267 if ctx.force { 268 doCheckpoint = true 269 } else { 270 r.globalPolicy.Add(1) 271 if r.globalPolicy.Check() { 272 doCheckpoint = true 273 } 274 } 275 if doCheckpoint { 276 now := time.Now() 277 entry, err := r.doGlobalCheckpoint(ctx.end, ctx.interval) 278 if err != nil { 279 logutil.Errorf("Global checkpoint %v failed: %v", entry, err) 280 continue 281 } 282 if err := r.saveCheckpoint(entry.start, entry.end); err != nil { 283 logutil.Errorf("Global checkpoint %v failed: %v", entry, err) 284 continue 285 } 286 logutil.Infof("%s is done, takes %s", entry.String(), time.Since(now)) 287 r.globalPolicy.Reset() 288 } 289 } 290 } 291 292 func (r *runner) onGCCheckpointEntries(items ...any) { 293 gcTS, needGC := r.getTSTOGC() 294 if !needGC { 295 return 296 } 297 r.gcCheckpointEntries(gcTS) 298 } 299 300 func (r *runner) getTSTOGC() (ts types.TS, needGC bool) { 301 ts = r.getGCTS() 302 if ts.IsEmpty() { 303 return 304 } 305 tsTOGC := r.getTSToGC() 306 if tsTOGC.Less(ts) { 307 ts = tsTOGC 308 } 309 gcedTS := r.getGCedTS() 310 if gcedTS.GreaterEq(ts) { 311 return 312 } 313 needGC = true 314 return 315 } 316 317 func (r *runner) gcCheckpointEntries(ts types.TS) { 318 if ts.IsEmpty() { 319 return 320 } 321 incrementals := r.GetAllIncrementalCheckpoints() 322 for _, incremental := range incrementals { 323 if incremental.LessEq(ts) { 324 err := incremental.GCEntry(r.fs) 325 if err != nil { 326 logutil.Warnf("gc %v failed: %v", incremental.String(), err) 327 panic(err) 328 } 329 err = incremental.GCMetadata(r.fs) 330 if err != nil { 331 panic(err) 332 } 333 r.DeleteIncrementalEntry(incremental) 334 } 335 } 336 globals := r.GetAllGlobalCheckpoints() 337 for _, global := range globals { 338 if global.LessEq(ts) { 339 err := global.GCEntry(r.fs) 340 if err != nil { 341 panic(err) 342 } 343 err = global.GCMetadata(r.fs) 344 if err != nil { 345 panic(err) 346 } 347 r.DeleteGlobalEntry(global) 348 } 349 } 350 } 351 352 func (r *runner) onIncrementalCheckpointEntries(items ...any) { 353 now := time.Now() 354 entry := r.MaxCheckpoint() 355 // In some unit tests, ckp is managed manually, and ckp deletiton (CleanPenddingCheckpoint) 356 // can be called when the queue still has unexecuted task. 357 // Add `entry == nil` here as protective codes 358 if entry == nil || entry.GetState() != ST_Running { 359 return 360 } 361 err := r.doIncrementalCheckpoint(entry) 362 if err != nil { 363 logutil.Errorf("Do checkpoint %s: %v", entry.String(), err) 364 return 365 } 366 entry.SetState(ST_Finished) 367 if err = r.saveCheckpoint(entry.start, entry.end); err != nil { 368 logutil.Errorf("Save checkpoint %s: %v", entry.String(), err) 369 return 370 } 371 372 lsn := r.source.GetMaxLSN(entry.start, entry.end) 373 e, err := r.wal.RangeCheckpoint(1, lsn) 374 if err != nil { 375 panic(err) 376 } 377 if err = e.WaitDone(); err != nil { 378 panic(err) 379 } 380 381 logutil.Infof("%s is done, takes %s, truncate %d", entry.String(), time.Since(now), lsn) 382 383 r.postCheckpointQueue.Enqueue(entry) 384 r.globalCheckpointQueue.Enqueue(&globalCheckpointContext{end: entry.end, interval: r.options.globalVersionInterval}) 385 } 386 387 func (r *runner) DeleteIncrementalEntry(entry *CheckpointEntry) { 388 r.storage.Lock() 389 defer r.storage.Unlock() 390 r.storage.entries.Delete(entry) 391 } 392 func (r *runner) DeleteGlobalEntry(entry *CheckpointEntry) { 393 r.storage.Lock() 394 defer r.storage.Unlock() 395 r.storage.globals.Delete(entry) 396 } 397 func (r *runner) FlushTable(dbID, tableID uint64, ts types.TS) (err error) { 398 makeCtx := func() *DirtyCtx { 399 tree := r.source.ScanInRangePruned(types.TS{}, ts) 400 tree.GetTree().Compact() 401 tableTree := tree.GetTree().GetTable(tableID) 402 if tableTree == nil { 403 return nil 404 } 405 nTree := common.NewTree() 406 nTree.Tables[tableID] = tableTree 407 entry := logtail.NewDirtyTreeEntry(types.TS{}, ts, nTree) 408 dirtyCtx := new(DirtyCtx) 409 dirtyCtx.tree = entry 410 dirtyCtx.force = true 411 return dirtyCtx 412 } 413 414 op := func() (ok bool, err error) { 415 dirtyCtx := makeCtx() 416 if dirtyCtx == nil { 417 return true, nil 418 } 419 if _, err = r.dirtyEntryQueue.Enqueue(dirtyCtx); err != nil { 420 return true, nil 421 } 422 return false, nil 423 } 424 425 err = common.RetryWithIntervalAndTimeout( 426 op, 427 r.options.forceFlushTimeout, 428 r.options.forceFlushCheckInterval, true) 429 if moerr.IsMoErrCode(err, moerr.ErrInternal) || moerr.IsMoErrCode(err, moerr.OkExpectedEOB) { 430 logutil.Warnf("Flush %d-%d :%v", dbID, tableID, err) 431 return nil 432 } 433 return 434 } 435 436 func (r *runner) saveCheckpoint(start, end types.TS) (err error) { 437 bat := r.collectCheckpointMetadata(start, end) 438 name := blockio.EncodeCheckpointMetadataFileName(CheckpointDir, PrefixMetadata, start, end) 439 writer := blockio.NewWriter(context.Background(), r.fs, name) 440 if _, err = writer.WriteBlock(bat); err != nil { 441 return 442 } 443 444 // TODO: checkpoint entry should maintain the location 445 _, err = writer.Sync() 446 return 447 } 448 449 func (r *runner) doIncrementalCheckpoint(entry *CheckpointEntry) (err error) { 450 factory := logtail.IncrementalCheckpointDataFactory(entry.start, entry.end) 451 data, err := factory(r.catalog) 452 if err != nil { 453 return 454 } 455 defer data.Close() 456 457 filename := uuid.NewString() 458 writer := blockio.NewWriter(context.Background(), r.fs, filename) 459 blks, err := data.WriteTo(writer) 460 if err != nil { 461 return 462 } 463 location := blockio.EncodeMetalocFromMetas(filename, blks) 464 entry.SetLocation(location) 465 return 466 } 467 468 func (r *runner) doGlobalCheckpoint(end types.TS, interval time.Duration) (entry *CheckpointEntry, err error) { 469 entry = NewCheckpointEntry(types.TS{}, end.Next(), ET_Global) 470 factory := logtail.GlobalCheckpointDataFactory(entry.end, interval) 471 data, err := factory(r.catalog) 472 if err != nil { 473 return 474 } 475 defer data.Close() 476 477 filename := uuid.NewString() 478 writer := blockio.NewWriter(context.Background(), r.fs, filename) 479 blks, err := data.WriteTo(writer) 480 if err != nil { 481 return 482 } 483 location := blockio.EncodeMetalocFromMetas(filename, blks) 484 entry.SetLocation(location) 485 r.tryAddNewGlobalCheckpointEntry(entry) 486 entry.SetState(ST_Finished) 487 return 488 } 489 490 func (r *runner) onPostCheckpointEntries(entries ...any) { 491 for _, e := range entries { 492 entry := e.(*CheckpointEntry) 493 494 // 1. broadcast event 495 r.observers.OnNewCheckpoint(entry.GetEnd()) 496 497 // TODO: 498 // 2. remove previous checkpoint 499 500 logutil.Debugf("Post %s", entry.String()) 501 } 502 } 503 504 func (r *runner) tryAddNewGlobalCheckpointEntry(entry *CheckpointEntry) (success bool) { 505 r.storage.Lock() 506 defer r.storage.Unlock() 507 r.storage.globals.Set(entry) 508 return true 509 } 510 511 func (r *runner) tryAddNewIncrementalCheckpointEntry(entry *CheckpointEntry) (success bool) { 512 r.storage.Lock() 513 defer r.storage.Unlock() 514 maxEntry, _ := r.storage.entries.Max() 515 516 // if it's the first entry, add it 517 if maxEntry == nil { 518 r.storage.entries.Set(entry) 519 success = true 520 return 521 } 522 523 // if it is not the right candidate, skip this request 524 // [startTs, endTs] --> [endTs+1, ?] 525 if !maxEntry.GetEnd().Next().Equal(entry.GetStart()) { 526 success = false 527 return 528 } 529 530 // if the max entry is not finished, skip this request 531 if !maxEntry.IsFinished() { 532 success = false 533 return 534 } 535 536 r.storage.entries.Set(entry) 537 538 success = true 539 return 540 } 541 542 func (r *runner) tryScheduleIncrementalCheckpoint(start types.TS) { 543 ts := types.BuildTS(time.Now().UTC().UnixNano(), 0) 544 _, count := r.source.ScanInRange(start, ts) 545 if count < r.options.minCount { 546 return 547 } 548 entry := NewCheckpointEntry(start, ts, ET_Incremental) 549 r.tryAddNewIncrementalCheckpointEntry(entry) 550 } 551 552 func (r *runner) tryScheduleCheckpoint() { 553 if r.disabled.Load() { 554 return 555 } 556 entry := r.MaxCheckpoint() 557 global := r.MaxGlobalCheckpoint() 558 559 // no prev checkpoint found. try schedule the first 560 // checkpoint 561 if entry == nil { 562 if global == nil { 563 r.tryScheduleIncrementalCheckpoint(types.TS{}) 564 return 565 } else { 566 maxTS := global.end.Prev() 567 if r.incrementalPolicy.Check(maxTS) { 568 r.tryScheduleIncrementalCheckpoint(maxTS.Next()) 569 } 570 return 571 } 572 } 573 574 if entry.IsPendding() { 575 check := func() (done bool) { 576 tree := r.source.ScanInRangePruned(entry.GetStart(), entry.GetEnd()) 577 tree.GetTree().Compact() 578 if tree.IsEmpty() { 579 done = true 580 } 581 return 582 } 583 584 if !check() { 585 logutil.Debugf("%s is waiting", entry.String()) 586 return 587 } 588 entry.SetState(ST_Running) 589 r.incrementalCheckpointQueue.Enqueue(struct{}{}) 590 return 591 } 592 593 if entry.IsRunning() { 594 r.incrementalCheckpointQueue.Enqueue(struct{}{}) 595 return 596 } 597 598 if r.incrementalPolicy.Check(entry.end) { 599 r.tryScheduleIncrementalCheckpoint(entry.end.Next()) 600 } 601 } 602 603 func (r *runner) fillDefaults() { 604 if r.options.forceFlushTimeout <= 0 { 605 r.options.forceFlushTimeout = time.Second * 30 606 } 607 if r.options.forceFlushCheckInterval <= 0 { 608 r.options.forceFlushCheckInterval = time.Millisecond * 400 609 } 610 if r.options.collectInterval <= 0 { 611 // TODO: define default value 612 r.options.collectInterval = time.Second * 5 613 } 614 if r.options.dirtyEntryQueueSize <= 0 { 615 r.options.dirtyEntryQueueSize = 10000 616 } 617 if r.options.waitQueueSize <= 1000 { 618 r.options.waitQueueSize = 1000 619 } 620 if r.options.checkpointQueueSize <= 1000 { 621 r.options.checkpointQueueSize = 1000 622 } 623 if r.options.minIncrementalInterval <= 0 { 624 r.options.minIncrementalInterval = time.Minute 625 } 626 if r.options.globalMinCount <= 0 { 627 r.options.globalMinCount = 10 628 } 629 if r.options.minCount <= 0 { 630 r.options.minCount = 10000 631 } 632 } 633 634 func (r *runner) tryCompactBlock(dbID, tableID, segmentID, id uint64, force bool) (err error) { 635 db, err := r.catalog.GetDatabaseByID(dbID) 636 if err != nil { 637 panic(err) 638 } 639 table, err := db.GetTableEntryByID(tableID) 640 if err != nil { 641 panic(err) 642 } 643 segment, err := table.GetSegmentByID(segmentID) 644 if err != nil { 645 panic(err) 646 } 647 blk, err := segment.GetBlockEntryByID(id) 648 if err != nil { 649 panic(err) 650 } 651 blkData := blk.GetBlockData() 652 score := blkData.EstimateScore(r.options.maxFlushInterval, force) 653 logutil.Debugf("%s [SCORE=%d]", blk.String(), score) 654 if score < 100 { 655 return 656 } 657 658 factory, taskType, scopes, err := blkData.BuildCompactionTaskFactory() 659 if err != nil || factory == nil { 660 logutil.Warnf("%s: %v", blkData.MutationInfo(), err) 661 return nil 662 } 663 664 if _, err = r.scheduler.ScheduleMultiScopedTxnTask(nil, taskType, scopes, factory); err != nil { 665 logutil.Warnf("%s: %v", blkData.MutationInfo(), err) 666 } 667 668 // always return nil 669 return nil 670 } 671 672 func (r *runner) onWaitWaitableItems(items ...any) { 673 // TODO: change for more waitable items 674 start := time.Now() 675 for _, item := range items { 676 ckpEntry := item.(wal.LogEntry) 677 err := ckpEntry.WaitDone() 678 if err != nil { 679 panic(err) 680 } 681 ckpEntry.Free() 682 } 683 logutil.Debugf("Total [%d] WAL Checkpointed | [%s]", len(items), time.Since(start)) 684 } 685 686 func (r *runner) tryCompactTree(entry *logtail.DirtyTreeEntry, force bool) { 687 if entry.IsEmpty() { 688 return 689 } 690 logutil.Debugf(entry.String()) 691 visitor := new(common.BaseTreeVisitor) 692 visitor.BlockFn = func(force bool) func(uint64, uint64, uint64, uint64) error { 693 return func(dbID, tableID, segmentID, id uint64) (err error) { 694 return r.tryCompactBlock(dbID, tableID, segmentID, id, force) 695 } 696 }(force) 697 698 if err := entry.GetTree().Visit(visitor); err != nil { 699 panic(err) 700 } 701 } 702 703 func (r *runner) onDirtyEntries(entries ...any) { 704 normal := logtail.NewEmptyDirtyTreeEntry() 705 force := logtail.NewEmptyDirtyTreeEntry() 706 for _, entry := range entries { 707 e := entry.(*DirtyCtx) 708 if e.force { 709 force.Merge(e.tree) 710 } else { 711 normal.Merge(e.tree) 712 } 713 } 714 if !force.IsEmpty() { 715 r.tryCompactTree(force, true) 716 } 717 718 if !normal.IsEmpty() { 719 r.tryCompactTree(normal, false) 720 } 721 } 722 723 func (r *runner) crontask(ctx context.Context) { 724 hb := w.NewHeartBeaterWithFunc(r.options.collectInterval, func() { 725 r.source.Run() 726 entry := r.source.GetAndRefreshMerged() 727 if entry.IsEmpty() { 728 logutil.Debugf("No dirty block found") 729 } else { 730 e := new(DirtyCtx) 731 e.tree = entry 732 r.dirtyEntryQueue.Enqueue(e) 733 } 734 r.tryScheduleCheckpoint() 735 }, nil) 736 hb.Start() 737 <-ctx.Done() 738 hb.Stop() 739 } 740 741 func (r *runner) EnqueueWait(item any) (err error) { 742 _, err = r.waitQueue.Enqueue(item) 743 return 744 } 745 746 func (r *runner) Start() { 747 r.onceStart.Do(func() { 748 r.postCheckpointQueue.Start() 749 r.incrementalCheckpointQueue.Start() 750 r.globalCheckpointQueue.Start() 751 r.gcCheckpointQueue.Start() 752 r.dirtyEntryQueue.Start() 753 r.waitQueue.Start() 754 if err := r.stopper.RunNamedTask("dirty-collector-job", r.crontask); err != nil { 755 panic(err) 756 } 757 }) 758 } 759 760 func (r *runner) Stop() { 761 r.onceStop.Do(func() { 762 r.stopper.Stop() 763 r.dirtyEntryQueue.Stop() 764 r.incrementalCheckpointQueue.Stop() 765 r.globalCheckpointQueue.Stop() 766 r.gcCheckpointQueue.Stop() 767 r.postCheckpointQueue.Stop() 768 r.waitQueue.Stop() 769 }) 770 } 771 772 func (r *runner) GetDirtyCollector() logtail.Collector { 773 return r.source 774 } 775 776 func (r *runner) CollectCheckpointsInRange(ctx context.Context, start, end types.TS) (locations string, checkpointed types.TS, err error) { 777 if r.IsTSStale(end) { 778 return "", types.TS{}, moerr.NewInternalError(ctx, "ts %v is staled", end.ToString()) 779 } 780 r.storage.Lock() 781 tree := r.storage.entries.Copy() 782 global, _ := r.storage.globals.Max() 783 r.storage.Unlock() 784 locs := make([]string, 0) 785 newStart := start 786 if global != nil && global.HasOverlap(start, end) { 787 locs = append(locs, global.GetLocation()) 788 newStart = global.end.Next() 789 checkpointed = global.GetEnd() 790 } 791 pivot := NewCheckpointEntry(newStart, newStart, ET_Incremental) 792 793 // For debug 794 // checkpoints := make([]*CheckpointEntry, 0) 795 // defer func() { 796 // items := tree.Items() 797 // logutil.Infof("CollectCheckpointsInRange: Pivot: %s", pivot.String()) 798 // for i, item := range items { 799 // logutil.Infof("CollectCheckpointsInRange: Source[%d]: %s", i, item.String()) 800 // } 801 // for i, ckp := range checkpoints { 802 // logutil.Infof("CollectCheckpointsInRange: Found[%d]:%s", i, ckp.String()) 803 // } 804 // logutil.Infof("CollectCheckpointsInRange: Checkpointed=%s", checkpointed.ToString()) 805 // }() 806 807 iter := tree.Iter() 808 defer iter.Release() 809 810 if ok := iter.Seek(pivot); ok { 811 if ok = iter.Prev(); ok { 812 e := iter.Item() 813 if !e.IsCommitted() { 814 if len(locs) == 0 { 815 return 816 } 817 locations = strings.Join(locs, ";") 818 return 819 } 820 if e.HasOverlap(newStart, end) { 821 locs = append(locs, e.GetLocation()) 822 checkpointed = e.GetEnd() 823 // checkpoints = append(checkpoints, e) 824 } 825 iter.Next() 826 } 827 for { 828 e := iter.Item() 829 if !e.IsCommitted() || !e.HasOverlap(newStart, end) { 830 break 831 } 832 locs = append(locs, e.GetLocation()) 833 checkpointed = e.GetEnd() 834 // checkpoints = append(checkpoints, e) 835 if ok = iter.Next(); !ok { 836 break 837 } 838 } 839 } else { 840 // if it is empty, quick quit 841 if ok = iter.Last(); !ok { 842 if len(locs) == 0 { 843 return 844 } 845 locations = strings.Join(locs, ";") 846 return 847 } 848 // get last entry 849 e := iter.Item() 850 // if it is committed and visible, quick quit 851 if !e.IsCommitted() || !e.HasOverlap(newStart, end) { 852 if len(locs) == 0 { 853 return 854 } 855 locations = strings.Join(locs, ";") 856 return 857 } 858 locs = append(locs, e.GetLocation()) 859 checkpointed = e.GetEnd() 860 // checkpoints = append(checkpoints, e) 861 } 862 863 if len(locs) == 0 { 864 return 865 } 866 locations = strings.Join(locs, ";") 867 return 868 }