github.com/matrixorigin/matrixone@v1.2.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 "bytes" 19 "context" 20 "fmt" 21 "math/rand" 22 "slices" 23 "strconv" 24 "strings" 25 "sync" 26 "sync/atomic" 27 "time" 28 29 v2 "github.com/matrixorigin/matrixone/pkg/util/metric/v2" 30 "go.uber.org/zap" 31 32 "github.com/matrixorigin/matrixone/pkg/util/fault" 33 34 "github.com/matrixorigin/matrixone/pkg/perfcounter" 35 36 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 37 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/dbutils" 38 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/model" 39 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tables/jobs" 40 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks" 41 42 "github.com/matrixorigin/matrixone/pkg/common/moerr" 43 "github.com/matrixorigin/matrixone/pkg/common/stopper" 44 "github.com/matrixorigin/matrixone/pkg/container/types" 45 "github.com/matrixorigin/matrixone/pkg/logutil" 46 "github.com/matrixorigin/matrixone/pkg/objectio" 47 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" 48 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/catalog" 49 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 50 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" 51 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" 52 w "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks/worker" 53 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/wal" 54 "github.com/tidwall/btree" 55 ) 56 57 type timeBasedPolicy struct { 58 interval time.Duration 59 } 60 61 func (p *timeBasedPolicy) Check(last types.TS) bool { 62 physical := last.Physical() 63 return physical <= time.Now().UTC().UnixNano()-p.interval.Nanoseconds() 64 } 65 66 type countBasedPolicy struct { 67 minCount int 68 } 69 70 func (p *countBasedPolicy) Check(current int) bool { 71 return current >= p.minCount 72 } 73 74 type globalCheckpointContext struct { 75 force bool 76 end types.TS 77 interval time.Duration 78 truncateLSN uint64 79 ckpLSN uint64 80 } 81 82 type tableAndSize struct { 83 tbl *catalog.TableEntry 84 asize int 85 dsize int 86 } 87 88 // Q: What does runner do? 89 // A: A checkpoint runner organizes and manages all checkpoint-related behaviors. It roughly 90 // does the following things: 91 // - Manage the life cycle of all checkpoints and provide some query interfaces. 92 // - A cron job periodically collects and analyzes dirty blocks, and flushes eligibl dirty 93 // blocks to the remote storage 94 // - The cron job peridically test whether a new checkpoint can be created. If it is not 95 // satisfied, it will wait for next trigger. Otherwise, it will start the process of 96 // creating a checkpoint. 97 98 // Q: How to collect dirty blocks? 99 // A: There is a logtail manager maintains all transaction information that occurred over a 100 // period of time. When a checkpoint is generated, we clean up the data before the 101 // checkpoint timestamp in logtail. 102 // 103 // |----to prune----| Time 104 // -----+----------------+-------------------------------------+-----------> 105 // t1 checkpoint-t10 t100 106 // 107 // For each transaction, it maintains a dirty block list. 108 // 109 // [t1]: TB1-[1] 110 // [t2]: TB2-[2] 111 // [t3]: TB1-[1],TB2-[3] 112 // [t4]: [] 113 // [t5]: TB2[3,4] 114 // ..... 115 // ..... 116 // When collecting the dirty blocks in [t1, t5], it will get 2 block list, which is represented 117 // with `common.Tree` 118 // [t1,t5] - - - - - - - - - <DirtyTreeEntry> 119 // / \ 120 // [TB1] [TB2] 121 // | | 122 // [1] [2,3,4] - - - - - - - leaf nodes are all dirty blocks 123 // We store the dirty tree entries into the internal storage. Over time, we'll see something like 124 // this inside the storage: 125 // - Entry[t1, t5] 126 // - Entry[t6, t12] 127 // - Entry[t13,t29] 128 // - Entry[t30,t47] 129 // ..... 130 // ..... 131 // When collecting the dirty blocks in [t1, t20], it will get 3 dirty trees from [t1,t5],[t6,t12], 132 // [t13,t29] and merge the three trees into a tree with all the leaf nodes being dirty blocks. 133 // 134 // In order to reduce the workload of scan, we have always been incremental scan. And also we will 135 // continue to clean up the entries in the storage. 136 137 // Q: How to test whether a block need to be flushed? 138 // A: It is an open question. There are a lot of options, just chose a simple strategy for now. 139 // Must: 140 // - The born transaction of the block was committed 141 // - No uncommitted transaction on the block 142 // Factors: 143 // - Max rows reached 144 // - Delete ratio 145 // - Max flush timeout 146 147 // Q: How to do incremental checkpoint? 148 // A: 1. Decide a checkpoint timestamp 149 // 2. Wait all transactions before timestamp were committed 150 // 3. Wait all dirty blocks before the timestamp were flushed 151 // 4. Prepare checkpoint data 152 // 5. Persist the checkpoint data 153 // 6. Persist the checkpoint meta data 154 // 7. Notify checkpoint events to all the observers 155 // 8. Schedule to remove stale checkpoint meta objects 156 157 // Q: How to boot from the checkpoints? 158 // A: When a meta version is created, it contains all information of the previouse version. So we always 159 // 160 // delete the stale versions when a new version is created. Over time, the number of objects under 161 // `ckp/` is small. 162 // 1. List all meta objects under `ckp/`. Get the latest meta object and read all checkpoint informations 163 // from the meta object. 164 // 2. Apply the latest global checkpoint 165 // 3. Apply the incremental checkpoint start from the version right after the global checkpoint to the 166 // latest version. 167 type runner struct { 168 options struct { 169 // checkpoint scanner interval duration 170 collectInterval time.Duration 171 172 // maximum dirty block flush interval duration 173 maxFlushInterval time.Duration 174 175 // minimum incremental checkpoint interval duration 176 minIncrementalInterval time.Duration 177 178 // minimum global checkpoint interval duration 179 globalMinCount int 180 forceUpdateGlobalInterval bool 181 globalVersionInterval time.Duration 182 183 // minimum count of uncheckpointed transactions allowed before the next checkpoint 184 minCount int 185 186 forceFlushTimeout time.Duration 187 forceFlushCheckInterval time.Duration 188 189 dirtyEntryQueueSize int 190 waitQueueSize int 191 checkpointQueueSize int 192 193 checkpointBlockRows int 194 checkpointSize int 195 196 reservedWALEntryCount uint64 197 } 198 199 ctx context.Context 200 201 // logtail sourcer 202 source logtail.Collector 203 catalog *catalog.Catalog 204 rt *dbutils.Runtime 205 observers *observers 206 wal wal.Driver 207 disabled atomic.Bool 208 209 stopper *stopper.Stopper 210 211 // memory storage of the checkpoint entries 212 storage struct { 213 sync.RWMutex 214 entries *btree.BTreeG[*CheckpointEntry] 215 globals *btree.BTreeG[*CheckpointEntry] 216 } 217 218 gcTS atomic.Value 219 220 // checkpoint policy 221 incrementalPolicy *timeBasedPolicy 222 globalPolicy *countBasedPolicy 223 224 dirtyEntryQueue sm.Queue 225 waitQueue sm.Queue 226 incrementalCheckpointQueue sm.Queue 227 globalCheckpointQueue sm.Queue 228 postCheckpointQueue sm.Queue 229 gcCheckpointQueue sm.Queue 230 231 objMemSizeList []tableAndSize 232 233 onceStart sync.Once 234 onceStop sync.Once 235 } 236 237 func NewRunner( 238 ctx context.Context, 239 rt *dbutils.Runtime, 240 catalog *catalog.Catalog, 241 source logtail.Collector, 242 wal wal.Driver, 243 opts ...Option) *runner { 244 r := &runner{ 245 ctx: ctx, 246 rt: rt, 247 catalog: catalog, 248 source: source, 249 observers: new(observers), 250 wal: wal, 251 } 252 r.storage.entries = btree.NewBTreeGOptions(func(a, b *CheckpointEntry) bool { 253 return a.end.Less(&b.end) 254 }, btree.Options{ 255 NoLocks: true, 256 }) 257 r.storage.globals = btree.NewBTreeGOptions(func(a, b *CheckpointEntry) bool { 258 return a.end.Less(&b.end) 259 }, btree.Options{ 260 NoLocks: true, 261 }) 262 for _, opt := range opts { 263 opt(r) 264 } 265 r.fillDefaults() 266 267 r.incrementalPolicy = &timeBasedPolicy{interval: r.options.minIncrementalInterval} 268 r.globalPolicy = &countBasedPolicy{minCount: r.options.globalMinCount} 269 r.stopper = stopper.NewStopper("CheckpointRunner") 270 r.dirtyEntryQueue = sm.NewSafeQueue(r.options.dirtyEntryQueueSize, 100, r.onDirtyEntries) 271 r.waitQueue = sm.NewSafeQueue(r.options.waitQueueSize, 100, r.onWaitWaitableItems) 272 r.incrementalCheckpointQueue = sm.NewSafeQueue(r.options.checkpointQueueSize, 100, r.onIncrementalCheckpointEntries) 273 r.globalCheckpointQueue = sm.NewSafeQueue(r.options.checkpointQueueSize, 100, r.onGlobalCheckpointEntries) 274 r.gcCheckpointQueue = sm.NewSafeQueue(100, 100, r.onGCCheckpointEntries) 275 r.postCheckpointQueue = sm.NewSafeQueue(1000, 1, r.onPostCheckpointEntries) 276 return r 277 } 278 279 func (r *runner) String() string { 280 var buf bytes.Buffer 281 _, _ = fmt.Fprintf(&buf, "CheckpointRunner<") 282 _, _ = fmt.Fprintf(&buf, "collectInterval=%v, ", r.options.collectInterval) 283 _, _ = fmt.Fprintf(&buf, "maxFlushInterval=%v, ", r.options.maxFlushInterval) 284 _, _ = fmt.Fprintf(&buf, "minIncrementalInterval=%v, ", r.options.minIncrementalInterval) 285 _, _ = fmt.Fprintf(&buf, "globalMinCount=%v, ", r.options.globalMinCount) 286 _, _ = fmt.Fprintf(&buf, "globalVersionInterval=%v, ", r.options.globalVersionInterval) 287 _, _ = fmt.Fprintf(&buf, "minCount=%v, ", r.options.minCount) 288 _, _ = fmt.Fprintf(&buf, "forceFlushTimeout=%v, ", r.options.forceFlushTimeout) 289 _, _ = fmt.Fprintf(&buf, "forceFlushCheckInterval=%v, ", r.options.forceFlushCheckInterval) 290 _, _ = fmt.Fprintf(&buf, "dirtyEntryQueueSize=%v, ", r.options.dirtyEntryQueueSize) 291 _, _ = fmt.Fprintf(&buf, "waitQueueSize=%v, ", r.options.waitQueueSize) 292 _, _ = fmt.Fprintf(&buf, "checkpointQueueSize=%v, ", r.options.checkpointQueueSize) 293 _, _ = fmt.Fprintf(&buf, "checkpointBlockRows=%v, ", r.options.checkpointBlockRows) 294 _, _ = fmt.Fprintf(&buf, "checkpointSize=%v, ", r.options.checkpointSize) 295 _, _ = fmt.Fprintf(&buf, ">") 296 return buf.String() 297 } 298 299 // Only used in UT 300 func (r *runner) DebugUpdateOptions(opts ...Option) { 301 for _, opt := range opts { 302 opt(r) 303 } 304 } 305 306 func (r *runner) onGlobalCheckpointEntries(items ...any) { 307 for _, item := range items { 308 ctx := item.(*globalCheckpointContext) 309 doCheckpoint := false 310 if ctx.force { 311 doCheckpoint = true 312 } else { 313 entriesCount := r.GetPenddingIncrementalCount() 314 if r.globalPolicy.Check(entriesCount) { 315 doCheckpoint = true 316 } 317 } 318 if doCheckpoint { 319 if _, err := r.doGlobalCheckpoint( 320 ctx.end, ctx.ckpLSN, ctx.truncateLSN, ctx.interval, 321 ); err != nil { 322 continue 323 } 324 } 325 } 326 } 327 328 func (r *runner) onGCCheckpointEntries(items ...any) { 329 gcTS, needGC := r.getTSTOGC() 330 if !needGC { 331 return 332 } 333 r.gcCheckpointEntries(gcTS) 334 } 335 336 func (r *runner) getTSTOGC() (ts types.TS, needGC bool) { 337 ts = r.getGCTS() 338 if ts.IsEmpty() { 339 return 340 } 341 tsTOGC := r.getTSToGC() 342 if tsTOGC.Less(&ts) { 343 ts = tsTOGC 344 } 345 gcedTS := r.getGCedTS() 346 if gcedTS.GreaterEq(&ts) { 347 return 348 } 349 needGC = true 350 return 351 } 352 353 func (r *runner) gcCheckpointEntries(ts types.TS) { 354 if ts.IsEmpty() { 355 return 356 } 357 incrementals := r.GetAllIncrementalCheckpoints() 358 for _, incremental := range incrementals { 359 if incremental.LessEq(ts) { 360 r.DeleteIncrementalEntry(incremental) 361 } 362 } 363 globals := r.GetAllGlobalCheckpoints() 364 for _, global := range globals { 365 if global.LessEq(ts) { 366 r.DeleteGlobalEntry(global) 367 } 368 } 369 } 370 371 func (r *runner) onIncrementalCheckpointEntries(items ...any) { 372 now := time.Now() 373 entry := r.MaxCheckpoint() 374 // In some unit tests, ckp is managed manually, and ckp deletiton (CleanPenddingCheckpoint) 375 // can be called when the queue still has unexecuted task. 376 // Add `entry == nil` here as protective codes 377 if entry == nil || entry.GetState() != ST_Running { 378 return 379 } 380 var ( 381 err error 382 errPhase string 383 lsnToTruncate uint64 384 lsn uint64 385 fatal bool 386 fields []zap.Field 387 ) 388 now = time.Now() 389 390 logutil.Info( 391 "Checkpoint-Start", 392 zap.String("entry", entry.String()), 393 ) 394 395 defer func() { 396 if err != nil { 397 var logger func(msg string, fields ...zap.Field) 398 if fatal { 399 logger = logutil.Fatal 400 } else { 401 logger = logutil.Error 402 } 403 logger( 404 "Checkpoint-Error", 405 zap.String("entry", entry.String()), 406 zap.Error(err), 407 zap.String("phase", errPhase), 408 zap.Duration("cost", time.Since(now)), 409 ) 410 } else { 411 fields = append(fields, zap.Duration("cost", time.Since(now))) 412 fields = append(fields, zap.Uint64("truncate", lsnToTruncate)) 413 fields = append(fields, zap.Uint64("lsn", lsn)) 414 fields = append(fields, zap.Uint64("reserve", r.options.reservedWALEntryCount)) 415 fields = append(fields, zap.String("entry", entry.String())) 416 logutil.Info( 417 "Checkpoint-End", 418 fields..., 419 ) 420 } 421 }() 422 423 if fields, err = r.doIncrementalCheckpoint(entry); err != nil { 424 errPhase = "do-ckp" 425 return 426 } 427 428 lsn = r.source.GetMaxLSN(entry.start, entry.end) 429 if lsn > r.options.reservedWALEntryCount { 430 lsnToTruncate = lsn - r.options.reservedWALEntryCount 431 } 432 entry.SetLSN(lsn, lsnToTruncate) 433 entry.SetState(ST_Finished) 434 435 if err = r.saveCheckpoint( 436 entry.start, entry.end, lsn, lsnToTruncate, 437 ); err != nil { 438 errPhase = "save-ckp" 439 return 440 } 441 442 var logEntry wal.LogEntry 443 if logEntry, err = r.wal.RangeCheckpoint(1, lsnToTruncate); err != nil { 444 errPhase = "wal-ckp" 445 fatal = true 446 return 447 } 448 if err = logEntry.WaitDone(); err != nil { 449 errPhase = "wait-wal-ckp-done" 450 fatal = true 451 return 452 } 453 454 r.postCheckpointQueue.Enqueue(entry) 455 r.globalCheckpointQueue.Enqueue(&globalCheckpointContext{ 456 end: entry.end, 457 interval: r.options.globalVersionInterval, 458 ckpLSN: lsn, 459 truncateLSN: lsnToTruncate, 460 }) 461 } 462 463 func (r *runner) DeleteIncrementalEntry(entry *CheckpointEntry) { 464 r.storage.Lock() 465 defer r.storage.Unlock() 466 r.storage.entries.Delete(entry) 467 perfcounter.Update(r.ctx, func(counter *perfcounter.CounterSet) { 468 counter.TAE.CheckPoint.DeleteIncrementalEntry.Add(1) 469 }) 470 } 471 func (r *runner) DeleteGlobalEntry(entry *CheckpointEntry) { 472 r.storage.Lock() 473 defer r.storage.Unlock() 474 r.storage.globals.Delete(entry) 475 perfcounter.Update(r.ctx, func(counter *perfcounter.CounterSet) { 476 counter.TAE.CheckPoint.DeleteGlobalEntry.Add(1) 477 }) 478 } 479 func (r *runner) FlushTable(ctx context.Context, dbID, tableID uint64, ts types.TS) (err error) { 480 iarg, sarg, flush := fault.TriggerFault("flush_table_error") 481 if flush && (iarg == 0 || rand.Int63n(iarg) == 0) { 482 return moerr.NewInternalError(ctx, sarg) 483 } 484 makeCtx := func() *DirtyCtx { 485 tree := r.source.ScanInRangePruned(types.TS{}, ts) 486 tree.GetTree().Compact() 487 tableTree := tree.GetTree().GetTable(tableID) 488 if tableTree == nil { 489 return nil 490 } 491 nTree := model.NewTree() 492 nTree.Tables[tableID] = tableTree 493 entry := logtail.NewDirtyTreeEntry(types.TS{}, ts, nTree) 494 dirtyCtx := new(DirtyCtx) 495 dirtyCtx.tree = entry 496 dirtyCtx.force = true 497 return dirtyCtx 498 } 499 500 op := func() (ok bool, err error) { 501 dirtyCtx := makeCtx() 502 if dirtyCtx == nil { 503 return true, nil 504 } 505 if _, err = r.dirtyEntryQueue.Enqueue(dirtyCtx); err != nil { 506 return true, nil 507 } 508 return false, nil 509 } 510 511 err = common.RetryWithIntervalAndTimeout( 512 op, 513 r.options.forceFlushTimeout, 514 r.options.forceFlushCheckInterval, true) 515 if moerr.IsMoErrCode(err, moerr.ErrInternal) || moerr.IsMoErrCode(err, moerr.OkExpectedEOB) { 516 logutil.Warnf("Flush %d-%d :%v", dbID, tableID, err) 517 return nil 518 } 519 return 520 } 521 522 func (r *runner) saveCheckpoint(start, end types.TS, ckpLSN, truncateLSN uint64) (err error) { 523 bat := r.collectCheckpointMetadata(start, end, ckpLSN, truncateLSN) 524 defer bat.Close() 525 name := blockio.EncodeCheckpointMetadataFileName(CheckpointDir, PrefixMetadata, start, end) 526 writer, err := objectio.NewObjectWriterSpecial(objectio.WriterCheckpoint, name, r.rt.Fs.Service) 527 if err != nil { 528 return err 529 } 530 if _, err = writer.Write(containers.ToCNBatch(bat)); err != nil { 531 return 532 } 533 534 // TODO: checkpoint entry should maintain the location 535 _, err = writer.WriteEnd(r.ctx) 536 return 537 } 538 539 func (r *runner) doIncrementalCheckpoint(entry *CheckpointEntry) (fields []zap.Field, err error) { 540 factory := logtail.IncrementalCheckpointDataFactory(entry.start, entry.end, true, false) 541 data, err := factory(r.catalog) 542 if err != nil { 543 return 544 } 545 fields = data.ExportStats("") 546 defer data.Close() 547 cnLocation, tnLocation, _, err := data.WriteTo(r.rt.Fs.Service, r.options.checkpointBlockRows, r.options.checkpointSize) 548 if err != nil { 549 return 550 } 551 entry.SetLocation(cnLocation, tnLocation) 552 553 perfcounter.Update(r.ctx, func(counter *perfcounter.CounterSet) { 554 counter.TAE.CheckPoint.DoIncrementalCheckpoint.Add(1) 555 }) 556 return 557 } 558 559 func (r *runner) doCheckpointForBackup(entry *CheckpointEntry) (location string, err error) { 560 factory := logtail.BackupCheckpointDataFactory(entry.start, entry.end) 561 data, err := factory(r.catalog) 562 if err != nil { 563 return 564 } 565 defer data.Close() 566 cnLocation, tnLocation, _, err := data.WriteTo(r.rt.Fs.Service, r.options.checkpointBlockRows, r.options.checkpointSize) 567 if err != nil { 568 return 569 } 570 entry.SetLocation(cnLocation, tnLocation) 571 location = fmt.Sprintf("%s:%d:%s:%s:%s", cnLocation.String(), entry.GetVersion(), entry.end.ToString(), tnLocation.String(), entry.start.ToString()) 572 perfcounter.Update(r.ctx, func(counter *perfcounter.CounterSet) { 573 counter.TAE.CheckPoint.DoIncrementalCheckpoint.Add(1) 574 }) 575 return 576 } 577 578 func (r *runner) doGlobalCheckpoint( 579 end types.TS, ckpLSN, truncateLSN uint64, interval time.Duration, 580 ) (entry *CheckpointEntry, err error) { 581 var ( 582 errPhase string 583 fields []zap.Field 584 ) 585 now := time.Now() 586 587 entry = NewCheckpointEntry(types.TS{}, end.Next(), ET_Global) 588 entry.ckpLSN = ckpLSN 589 entry.truncateLSN = truncateLSN 590 591 logutil.Info( 592 "Checkpoint-Start", 593 zap.String("entry", entry.String()), 594 ) 595 596 defer func() { 597 if err != nil { 598 logutil.Error( 599 "Checkpoint-Error", 600 zap.String("entry", entry.String()), 601 zap.String("phase", errPhase), 602 zap.Error(err), 603 zap.Duration("cost", time.Since(now)), 604 ) 605 } else { 606 fields = append(fields, zap.Duration("cost", time.Since(now))) 607 fields = append(fields, zap.String("entry", entry.String())) 608 logutil.Info( 609 "Checkpoint-End", 610 fields..., 611 ) 612 } 613 }() 614 615 factory := logtail.GlobalCheckpointDataFactory(entry.end, interval) 616 data, err := factory(r.catalog) 617 if err != nil { 618 errPhase = "collect" 619 return 620 } 621 fields = data.ExportStats("") 622 defer data.Close() 623 624 cnLocation, tnLocation, _, err := data.WriteTo( 625 r.rt.Fs.Service, r.options.checkpointBlockRows, r.options.checkpointSize, 626 ) 627 if err != nil { 628 errPhase = "flush" 629 return 630 } 631 632 entry.SetLocation(cnLocation, tnLocation) 633 r.tryAddNewGlobalCheckpointEntry(entry) 634 entry.SetState(ST_Finished) 635 636 if err = r.saveCheckpoint(entry.start, entry.end, 0, 0); err != nil { 637 errPhase = "save" 638 return 639 } 640 641 perfcounter.Update(r.ctx, func(counter *perfcounter.CounterSet) { 642 counter.TAE.CheckPoint.DoGlobalCheckPoint.Add(1) 643 }) 644 return 645 } 646 647 func (r *runner) onPostCheckpointEntries(entries ...any) { 648 for _, e := range entries { 649 entry := e.(*CheckpointEntry) 650 651 // 1. broadcast event 652 r.observers.OnNewCheckpoint(entry.GetEnd()) 653 654 // TODO: 655 // 2. remove previous checkpoint 656 657 logutil.Debugf("Post %s", entry.String()) 658 } 659 } 660 661 func (r *runner) tryAddNewGlobalCheckpointEntry(entry *CheckpointEntry) (success bool) { 662 r.storage.Lock() 663 defer r.storage.Unlock() 664 r.storage.globals.Set(entry) 665 return true 666 } 667 668 func (r *runner) tryAddNewIncrementalCheckpointEntry(entry *CheckpointEntry) (success bool) { 669 r.storage.Lock() 670 defer r.storage.Unlock() 671 maxEntry, _ := r.storage.entries.Max() 672 673 // if it's the first entry, add it 674 if maxEntry == nil { 675 r.storage.entries.Set(entry) 676 success = true 677 return 678 } 679 680 // if it is not the right candidate, skip this request 681 // [startTs, endTs] --> [endTs+1, ?] 682 endTS := maxEntry.GetEnd() 683 startTS := entry.GetStart() 684 nextTS := endTS.Next() 685 if !nextTS.Equal(&startTS) { 686 success = false 687 return 688 } 689 690 // if the max entry is not finished, skip this request 691 if !maxEntry.IsFinished() { 692 success = false 693 return 694 } 695 696 r.storage.entries.Set(entry) 697 698 success = true 699 return 700 } 701 702 // Since there is no wal after recovery, the checkpoint lsn before backup must be set to 0. 703 func (r *runner) tryAddNewBackupCheckpointEntry(entry *CheckpointEntry) (success bool) { 704 entry.entryType = ET_Incremental 705 success = r.tryAddNewIncrementalCheckpointEntry(entry) 706 if !success { 707 return 708 } 709 r.storage.Lock() 710 defer r.storage.Unlock() 711 it := r.storage.entries.Iter() 712 for it.Next() { 713 e := it.Item() 714 e.ckpLSN = 0 715 e.truncateLSN = 0 716 } 717 return 718 } 719 720 func (r *runner) tryScheduleIncrementalCheckpoint(start, end types.TS) { 721 // ts := types.BuildTS(time.Now().UTC().UnixNano(), 0) 722 _, count := r.source.ScanInRange(start, end) 723 if count < r.options.minCount { 724 return 725 } 726 entry := NewCheckpointEntry(start, end, ET_Incremental) 727 r.tryAddNewIncrementalCheckpointEntry(entry) 728 } 729 730 func (r *runner) tryScheduleCheckpoint(endts types.TS) { 731 if r.disabled.Load() { 732 return 733 } 734 entry := r.MaxCheckpoint() 735 global := r.MaxGlobalCheckpoint() 736 737 // no prev checkpoint found. try schedule the first 738 // checkpoint 739 if entry == nil { 740 if global == nil { 741 r.tryScheduleIncrementalCheckpoint(types.TS{}, endts) 742 return 743 } else { 744 maxTS := global.end.Prev() 745 if r.incrementalPolicy.Check(maxTS) { 746 r.tryScheduleIncrementalCheckpoint(maxTS.Next(), endts) 747 } 748 return 749 } 750 } 751 752 if entry.IsPendding() { 753 check := func() (done bool) { 754 if !r.source.IsCommitted(entry.GetStart(), entry.GetEnd()) { 755 return false 756 } 757 tree := r.source.ScanInRangePruned(entry.GetStart(), entry.GetEnd()) 758 tree.GetTree().Compact() 759 if !tree.IsEmpty() && entry.CheckPrintTime() { 760 logutil.Infof("waiting for dirty tree %s", tree.String()) 761 entry.IncrWaterLine() 762 } 763 return tree.IsEmpty() 764 } 765 766 if !check() { 767 logutil.Debugf("%s is waiting", entry.String()) 768 return 769 } 770 entry.SetState(ST_Running) 771 v2.TaskCkpEntryPendingDurationHistogram.Observe(time.Since(entry.lastPrint).Seconds()) 772 r.incrementalCheckpointQueue.Enqueue(struct{}{}) 773 return 774 } 775 776 if entry.IsRunning() { 777 r.incrementalCheckpointQueue.Enqueue(struct{}{}) 778 return 779 } 780 781 if r.incrementalPolicy.Check(entry.end) { 782 r.tryScheduleIncrementalCheckpoint(entry.end.Next(), endts) 783 } 784 } 785 786 func (r *runner) fillDefaults() { 787 if r.options.forceFlushTimeout <= 0 { 788 r.options.forceFlushTimeout = time.Second * 90 789 } 790 if r.options.forceFlushCheckInterval <= 0 { 791 r.options.forceFlushCheckInterval = time.Millisecond * 400 792 } 793 if r.options.collectInterval <= 0 { 794 // TODO: define default value 795 r.options.collectInterval = time.Second * 5 796 } 797 if r.options.dirtyEntryQueueSize <= 0 { 798 r.options.dirtyEntryQueueSize = 10000 799 } 800 if r.options.waitQueueSize <= 1000 { 801 r.options.waitQueueSize = 1000 802 } 803 if r.options.checkpointQueueSize <= 1000 { 804 r.options.checkpointQueueSize = 1000 805 } 806 if r.options.minIncrementalInterval <= 0 { 807 r.options.minIncrementalInterval = time.Minute 808 } 809 if r.options.globalMinCount <= 0 { 810 r.options.globalMinCount = 10 811 } 812 if r.options.minCount <= 0 { 813 r.options.minCount = 10000 814 } 815 if r.options.checkpointBlockRows <= 0 { 816 r.options.checkpointBlockRows = logtail.DefaultCheckpointBlockRows 817 } 818 if r.options.checkpointSize <= 0 { 819 r.options.checkpointSize = logtail.DefaultCheckpointSize 820 } 821 } 822 823 func (r *runner) onWaitWaitableItems(items ...any) { 824 // TODO: change for more waitable items 825 start := time.Now() 826 for _, item := range items { 827 ckpEntry := item.(wal.LogEntry) 828 err := ckpEntry.WaitDone() 829 if err != nil { 830 panic(err) 831 } 832 ckpEntry.Free() 833 } 834 logutil.Debugf("Total [%d] WAL Checkpointed | [%s]", len(items), time.Since(start)) 835 } 836 837 func (r *runner) fireFlushTabletail(table *catalog.TableEntry, tree *model.TableTree, endTs types.TS) error { 838 metas := make([]*catalog.ObjectEntry, 0, 10) 839 for _, obj := range tree.Objs { 840 object, err := table.GetObjectByID(obj.ID) 841 if err != nil { 842 panic(err) 843 } 844 metas = append(metas, object) 845 } 846 847 // freeze all append 848 scopes := make([]common.ID, 0, len(metas)) 849 for _, meta := range metas { 850 if !meta.GetObjectData().PrepareCompact() { 851 logutil.Infof("[FlushTabletail] %d-%s / %s false prepareCompact ", table.ID, table.GetLastestSchemaLocked().Name, meta.ID.String()) 852 return moerr.GetOkExpectedEOB() 853 } 854 scopes = append(scopes, *meta.AsCommonID()) 855 } 856 857 factory := jobs.FlushTableTailTaskFactory(metas, r.rt, endTs) 858 if _, err := r.rt.Scheduler.ScheduleMultiScopedTxnTask(nil, tasks.DataCompactionTask, scopes, factory); err != nil { 859 if err != tasks.ErrScheduleScopeConflict { 860 logutil.Infof("[FlushTabletail] %d-%s %v", table.ID, table.GetLastestSchemaLocked().Name, err) 861 } 862 return moerr.GetOkExpectedEOB() 863 } 864 return nil 865 } 866 867 func (r *runner) EstimateTableMemSize(table *catalog.TableEntry, tree *model.TableTree) (asize int, dsize int) { 868 for _, obj := range tree.Objs { 869 object, err := table.GetObjectByID(obj.ID) 870 if err != nil { 871 panic(err) 872 } 873 a, d := object.GetObjectData().EstimateMemSize() 874 asize += a 875 dsize += d 876 } 877 return 878 } 879 880 func (r *runner) tryCompactTree(entry *logtail.DirtyTreeEntry, force bool) { 881 if entry.IsEmpty() { 882 return 883 } 884 logutil.Debugf(entry.String()) 885 886 r.objMemSizeList = r.objMemSizeList[:0] 887 sizevisitor := new(model.BaseTreeVisitor) 888 var totalSize, totalASize int 889 sizevisitor.TableFn = func(did, tid uint64) error { 890 db, err := r.catalog.GetDatabaseByID(did) 891 if err != nil { 892 panic(err) 893 } 894 table, err := db.GetTableEntryByID(tid) 895 if err != nil { 896 panic(err) 897 } 898 if !table.Stats.Inited { 899 table.Stats.Lock() 900 table.Stats.InitWithLock(r.options.maxFlushInterval) 901 table.Stats.Unlock() 902 } 903 dirtyTree := entry.GetTree().GetTable(tid) 904 asize, dsize := r.EstimateTableMemSize(table, dirtyTree) 905 totalSize += asize + dsize 906 totalASize += asize 907 r.objMemSizeList = append(r.objMemSizeList, tableAndSize{table, asize, dsize}) 908 return moerr.GetOkStopCurrRecur() 909 } 910 if err := entry.GetTree().Visit(sizevisitor); err != nil { 911 panic(err) 912 } 913 914 slices.SortFunc(r.objMemSizeList, func(a, b tableAndSize) int { 915 if a.asize > b.asize { 916 return -1 917 } else if a.asize < b.asize { 918 return 1 919 } else { 920 return 0 921 } 922 }) 923 924 pressure := float64(totalSize) / float64(common.RuntimeOverallFlushMemCap.Load()) 925 if pressure > 1.0 { 926 pressure = 1.0 927 } 928 count := 0 929 930 logutil.Infof("[flushtabletail] scan result: pressure %v, totalsize %v", pressure, common.HumanReadableBytes(totalSize)) 931 932 for _, ticket := range r.objMemSizeList { 933 table, asize, dsize := ticket.tbl, ticket.asize, ticket.dsize 934 dirtyTree := entry.GetTree().GetTable(table.ID) 935 _, endTs := entry.GetTimeRange() 936 937 stats := table.Stats 938 stats.Lock() 939 defer stats.Unlock() 940 941 if force { 942 logutil.Infof("[flushtabletail] force flush %v-%s", table.ID, table.GetLastestSchemaLocked().Name) 943 if err := r.fireFlushTabletail(table, dirtyTree, endTs); err == nil { 944 stats.ResetDeadlineWithLock() 945 } 946 continue 947 } 948 949 if stats.LastFlush.IsEmpty() { 950 // first boot, just bail out, and never enter this branch again 951 stats.LastFlush = stats.LastFlush.Next() 952 stats.ResetDeadlineWithLock() 953 continue 954 } 955 956 flushReady := func() bool { 957 if !table.IsActive() { 958 count++ 959 if pressure < 0.5 || count < 200 { 960 return true 961 } 962 return false 963 } 964 if stats.FlushDeadline.Before(time.Now()) { 965 return true 966 } 967 if asize+dsize > stats.FlushMemCapacity { 968 return true 969 } 970 if asize < common.Const1MBytes && dsize > 2*common.Const1MBytes+common.Const1MBytes/2 { 971 return true 972 } 973 if asize > common.Const1MBytes && rand.Float64() < pressure { 974 return true 975 } 976 return false 977 } 978 979 ready := flushReady() 980 // debug log, delete later 981 if !stats.LastFlush.IsEmpty() && asize+dsize > 2*1000*1024 { 982 logutil.Infof("[flushtabletail] %v(%v) %v dels FlushCountDown %v, flushReady %v", 983 table.GetLastestSchemaLocked().Name, 984 common.HumanReadableBytes(asize+dsize), 985 common.HumanReadableBytes(dsize), 986 time.Until(stats.FlushDeadline), 987 ready, 988 ) 989 } 990 991 if ready { 992 if err := r.fireFlushTabletail(table, dirtyTree, endTs); err == nil { 993 stats.ResetDeadlineWithLock() 994 } 995 } 996 } 997 } 998 999 func (r *runner) onDirtyEntries(entries ...any) { 1000 normal := logtail.NewEmptyDirtyTreeEntry() 1001 force := logtail.NewEmptyDirtyTreeEntry() 1002 for _, entry := range entries { 1003 e := entry.(*DirtyCtx) 1004 if e.force { 1005 force.Merge(e.tree) 1006 } else { 1007 normal.Merge(e.tree) 1008 } 1009 } 1010 if !force.IsEmpty() { 1011 r.tryCompactTree(force, true) 1012 } 1013 1014 if !normal.IsEmpty() { 1015 r.tryCompactTree(normal, false) 1016 } 1017 } 1018 1019 func (r *runner) crontask(ctx context.Context) { 1020 lag := 3 * time.Second 1021 if r.options.maxFlushInterval < time.Second { 1022 lag = 0 * time.Second 1023 } 1024 hb := w.NewHeartBeaterWithFunc(r.options.collectInterval, func() { 1025 r.source.Run(lag) 1026 entry := r.source.GetAndRefreshMerged() 1027 _, endts := entry.GetTimeRange() 1028 if entry.IsEmpty() { 1029 logutil.Debugf("[flushtabletail]No dirty block found") 1030 } else { 1031 e := new(DirtyCtx) 1032 e.tree = entry 1033 r.dirtyEntryQueue.Enqueue(e) 1034 } 1035 r.tryScheduleCheckpoint(endts) 1036 }, nil) 1037 hb.Start() 1038 <-ctx.Done() 1039 hb.Stop() 1040 } 1041 1042 func (r *runner) EnqueueWait(item any) (err error) { 1043 _, err = r.waitQueue.Enqueue(item) 1044 return 1045 } 1046 1047 func (r *runner) Start() { 1048 r.onceStart.Do(func() { 1049 r.postCheckpointQueue.Start() 1050 r.incrementalCheckpointQueue.Start() 1051 r.globalCheckpointQueue.Start() 1052 r.gcCheckpointQueue.Start() 1053 r.dirtyEntryQueue.Start() 1054 r.waitQueue.Start() 1055 if err := r.stopper.RunNamedTask("dirty-collector-job", r.crontask); err != nil { 1056 panic(err) 1057 } 1058 }) 1059 } 1060 1061 func (r *runner) Stop() { 1062 r.onceStop.Do(func() { 1063 r.stopper.Stop() 1064 r.dirtyEntryQueue.Stop() 1065 r.incrementalCheckpointQueue.Stop() 1066 r.globalCheckpointQueue.Stop() 1067 r.gcCheckpointQueue.Stop() 1068 r.postCheckpointQueue.Stop() 1069 r.waitQueue.Stop() 1070 }) 1071 } 1072 1073 func (r *runner) GetDirtyCollector() logtail.Collector { 1074 return r.source 1075 } 1076 1077 func (r *runner) CollectCheckpointsInRange(ctx context.Context, start, end types.TS) (locations string, checkpointed types.TS, err error) { 1078 if r.IsTSStale(end) { 1079 return "", types.TS{}, moerr.NewInternalError(ctx, "ts %v is staled", end.ToString()) 1080 } 1081 r.storage.Lock() 1082 tree := r.storage.entries.Copy() 1083 global, _ := r.storage.globals.Max() 1084 r.storage.Unlock() 1085 locs := make([]string, 0) 1086 ckpStart := types.MaxTs() 1087 newStart := start 1088 if global != nil && global.HasOverlap(start, end) { 1089 locs = append(locs, global.GetLocation().String()) 1090 locs = append(locs, strconv.Itoa(int(global.version))) 1091 newStart = global.end.Next() 1092 ckpStart = global.GetEnd() 1093 checkpointed = global.GetEnd() 1094 } 1095 pivot := NewCheckpointEntry(newStart, newStart, ET_Incremental) 1096 1097 // For debug 1098 // checkpoints := make([]*CheckpointEntry, 0) 1099 // defer func() { 1100 // items := tree.Items() 1101 // logutil.Infof("CollectCheckpointsInRange: Pivot: %s", pivot.String()) 1102 // for i, item := range items { 1103 // logutil.Infof("CollectCheckpointsInRange: Source[%d]: %s", i, item.String()) 1104 // } 1105 // for i, ckp := range checkpoints { 1106 // logutil.Infof("CollectCheckpointsInRange: Found[%d]:%s", i, ckp.String()) 1107 // } 1108 // logutil.Infof("CollectCheckpointsInRange: Checkpointed=%s", checkpointed.ToString()) 1109 // }() 1110 1111 iter := tree.Iter() 1112 defer iter.Release() 1113 1114 if ok := iter.Seek(pivot); ok { 1115 if ok = iter.Prev(); ok { 1116 e := iter.Item() 1117 if !e.IsCommitted() { 1118 if len(locs) == 0 { 1119 return 1120 } 1121 duration := fmt.Sprintf("[%s_%s]", 1122 ckpStart.ToString(), 1123 ckpStart.ToString()) 1124 locs = append(locs, duration) 1125 locations = strings.Join(locs, ";") 1126 return 1127 } 1128 if e.HasOverlap(newStart, end) { 1129 locs = append(locs, e.GetLocation().String()) 1130 locs = append(locs, strconv.Itoa(int(e.version))) 1131 start := e.GetStart() 1132 if start.Less(&ckpStart) { 1133 ckpStart = start 1134 } 1135 checkpointed = e.GetEnd() 1136 // checkpoints = append(checkpoints, e) 1137 } 1138 iter.Next() 1139 } 1140 for { 1141 e := iter.Item() 1142 if !e.IsCommitted() || !e.HasOverlap(newStart, end) { 1143 break 1144 } 1145 locs = append(locs, e.GetLocation().String()) 1146 locs = append(locs, strconv.Itoa(int(e.version))) 1147 start := e.GetStart() 1148 if start.Less(&ckpStart) { 1149 ckpStart = start 1150 } 1151 checkpointed = e.GetEnd() 1152 // checkpoints = append(checkpoints, e) 1153 if ok = iter.Next(); !ok { 1154 break 1155 } 1156 } 1157 } else { 1158 // if it is empty, quick quit 1159 if ok = iter.Last(); !ok { 1160 if len(locs) == 0 { 1161 return 1162 } 1163 duration := fmt.Sprintf("[%s_%s]", 1164 ckpStart.ToString(), 1165 ckpStart.ToString()) 1166 locs = append(locs, duration) 1167 locations = strings.Join(locs, ";") 1168 return 1169 } 1170 // get last entry 1171 e := iter.Item() 1172 // if it is committed and visible, quick quit 1173 if !e.IsCommitted() || !e.HasOverlap(newStart, end) { 1174 if len(locs) == 0 { 1175 return 1176 } 1177 duration := fmt.Sprintf("[%s_%s]", 1178 ckpStart.ToString(), 1179 ckpStart.ToString()) 1180 locs = append(locs, duration) 1181 locations = strings.Join(locs, ";") 1182 return 1183 } 1184 locs = append(locs, e.GetLocation().String()) 1185 locs = append(locs, strconv.Itoa(int(e.version))) 1186 start := e.GetStart() 1187 if start.Less(&ckpStart) { 1188 ckpStart = start 1189 } 1190 checkpointed = e.GetEnd() 1191 // checkpoints = append(checkpoints, e) 1192 } 1193 1194 if len(locs) == 0 { 1195 return 1196 } 1197 duration := fmt.Sprintf("[%s_%s]", 1198 ckpStart.ToString(), 1199 checkpointed.ToString()) 1200 locs = append(locs, duration) 1201 locations = strings.Join(locs, ";") 1202 return 1203 }