github.com/matrixorigin/matrixone@v1.2.0/pkg/vm/engine/tae/db/gc/checkpoint.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 gc 16 17 import ( 18 "context" 19 "github.com/matrixorigin/matrixone/pkg/common/moerr" 20 "github.com/matrixorigin/matrixone/pkg/common/mpool" 21 "github.com/matrixorigin/matrixone/pkg/container/types" 22 "github.com/matrixorigin/matrixone/pkg/fileservice" 23 "github.com/matrixorigin/matrixone/pkg/logutil" 24 "github.com/matrixorigin/matrixone/pkg/objectio" 25 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/blockio" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/containers" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/checkpoint" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" 30 "sync" 31 "sync/atomic" 32 "time" 33 ) 34 35 type checkpointCleaner struct { 36 fs *objectio.ObjectFS 37 ctx context.Context 38 39 // ckpClient is used to get the instance of the specified checkpoint 40 ckpClient checkpoint.RunnerReader 41 42 // maxConsumed is to mark which checkpoint the current DiskCleaner has processed, 43 // through which you can get the next checkpoint to be processed 44 maxConsumed atomic.Pointer[checkpoint.CheckpointEntry] 45 46 // minMerged is to mark at which checkpoint the full 47 // GCTable in the current DiskCleaner is generated, 48 // UT case needs to use 49 minMerged atomic.Pointer[checkpoint.CheckpointEntry] 50 51 maxCompared atomic.Pointer[checkpoint.CheckpointEntry] 52 53 ckpStage atomic.Pointer[types.TS] 54 ckpGC atomic.Pointer[types.TS] 55 56 // minMergeCount is the configuration of the merge GC metadata file. 57 // When the GC file is greater than or equal to minMergeCount, 58 // the merge GC metadata file will be triggered and the expired file will be deleted. 59 minMergeCount struct { 60 sync.RWMutex 61 count int 62 } 63 64 // inputs is to record the currently valid GCTable 65 inputs struct { 66 sync.RWMutex 67 tables []*GCTable 68 } 69 70 // outputs is a list of files that have been deleted 71 outputs struct { 72 sync.RWMutex 73 files []string 74 } 75 76 // checker is to check whether the checkpoint can be consumed 77 checker struct { 78 sync.RWMutex 79 extras []func(item any) bool 80 } 81 82 // delWorker is a worker that deletes s3‘s objects or local 83 // files, and only one worker will run 84 delWorker *GCWorker 85 86 disableGC bool 87 88 option struct { 89 sync.RWMutex 90 enableGC bool 91 } 92 93 snapshotMeta *logtail.SnapshotMeta 94 95 mPool *mpool.MPool 96 } 97 98 func NewCheckpointCleaner( 99 ctx context.Context, 100 fs *objectio.ObjectFS, 101 ckpClient checkpoint.RunnerReader, 102 disableGC bool, 103 ) Cleaner { 104 cleaner := &checkpointCleaner{ 105 ctx: ctx, 106 fs: fs, 107 ckpClient: ckpClient, 108 disableGC: disableGC, 109 } 110 cleaner.delWorker = NewGCWorker(fs, cleaner) 111 cleaner.minMergeCount.count = MinMergeCount 112 cleaner.snapshotMeta = logtail.NewSnapshotMeta() 113 cleaner.option.enableGC = true 114 cleaner.mPool = common.DebugAllocator 115 return cleaner 116 } 117 118 func (c *checkpointCleaner) Stop() { 119 } 120 121 func (c *checkpointCleaner) GetMPool() *mpool.MPool { 122 return c.mPool 123 } 124 125 func (c *checkpointCleaner) SetTid(tid uint64) { 126 c.snapshotMeta.Lock() 127 defer c.snapshotMeta.Unlock() 128 c.snapshotMeta.SetTid(tid) 129 } 130 131 func (c *checkpointCleaner) EnableGCForTest() { 132 c.option.Lock() 133 defer c.option.Unlock() 134 c.option.enableGC = true 135 } 136 137 func (c *checkpointCleaner) DisableGCForTest() { 138 c.option.Lock() 139 defer c.option.Unlock() 140 c.option.enableGC = false 141 } 142 143 func (c *checkpointCleaner) isEnableGC() bool { 144 c.option.Lock() 145 defer c.option.Unlock() 146 return c.option.enableGC 147 } 148 149 func (c *checkpointCleaner) Replay() error { 150 dirs, err := c.fs.ListDir(GCMetaDir) 151 if err != nil { 152 return err 153 } 154 if len(dirs) == 0 { 155 return nil 156 } 157 minMergedStart := types.TS{} 158 minMergedEnd := types.TS{} 159 maxConsumedStart := types.TS{} 160 maxConsumedEnd := types.TS{} 161 maxSnapEnd := types.TS{} 162 maxAcctEnd := types.TS{} 163 var fullGCFile fileservice.DirEntry 164 // Get effective minMerged 165 var snapFile, acctFile string 166 for _, dir := range dirs { 167 start, end, ext := blockio.DecodeGCMetadataFileName(dir.Name) 168 if ext == blockio.GCFullExt { 169 if minMergedStart.IsEmpty() || minMergedStart.Less(&start) { 170 minMergedStart = start 171 minMergedEnd = end 172 maxConsumedStart = start 173 maxConsumedEnd = end 174 fullGCFile = dir 175 } 176 } 177 if ext == blockio.SnapshotExt && maxSnapEnd.Less(&end) { 178 maxSnapEnd = end 179 snapFile = dir.Name 180 } 181 if ext == blockio.AcctExt && maxAcctEnd.Less(&end) { 182 maxAcctEnd = end 183 acctFile = dir.Name 184 } 185 } 186 readDirs := make([]fileservice.DirEntry, 0) 187 if !minMergedStart.IsEmpty() { 188 readDirs = append(readDirs, fullGCFile) 189 } 190 for _, dir := range dirs { 191 start, end, ext := blockio.DecodeGCMetadataFileName(dir.Name) 192 if ext == blockio.GCFullExt || ext == blockio.SnapshotExt || ext == blockio.AcctExt { 193 continue 194 } 195 if (maxConsumedStart.IsEmpty() || maxConsumedStart.Less(&end)) && 196 minMergedEnd.Less(&end) { 197 maxConsumedStart = start 198 maxConsumedEnd = end 199 readDirs = append(readDirs, dir) 200 } 201 } 202 if len(readDirs) == 0 { 203 return nil 204 } 205 for _, dir := range readDirs { 206 table := NewGCTable() 207 _, end, _ := blockio.DecodeGCMetadataFileName(dir.Name) 208 err = table.ReadTable(c.ctx, GCMetaDir+dir.Name, dir.Size, c.fs, end) 209 if err != nil { 210 return err 211 } 212 c.updateInputs(table) 213 } 214 if snapFile != "" { 215 err = c.snapshotMeta.ReadMeta(c.ctx, GCMetaDir+snapFile, c.fs.Service) 216 if err != nil { 217 return err 218 } 219 } 220 ckp := checkpoint.NewCheckpointEntry(maxConsumedStart, maxConsumedEnd, checkpoint.ET_Incremental) 221 c.updateMaxConsumed(ckp) 222 ckp = checkpoint.NewCheckpointEntry(minMergedStart, minMergedEnd, checkpoint.ET_Incremental) 223 c.updateMinMerged(ckp) 224 225 if acctFile != "" { 226 err = c.snapshotMeta.ReadTableInfo(c.ctx, GCMetaDir+acctFile, c.fs.Service) 227 if err != nil { 228 return err 229 } 230 } else { 231 //No account table information, it may be a new cluster or an upgraded cluster, 232 //and the table information needs to be initialized from the checkpoint 233 maxConsumed := c.maxConsumed.Load() 234 checkpointEntries, err := checkpoint.ListSnapshotCheckpoint(c.ctx, c.fs.Service, ckp.GetEnd(), 0, checkpoint.SpecifiedCheckpoint) 235 if err != nil { 236 logutil.Warnf("list checkpoint failed, err[%v]", err) 237 } 238 if len(checkpointEntries) == 0 { 239 return nil 240 } 241 for _, entry := range checkpointEntries { 242 logutil.Infof("load checkpoint: %s, consumedEnd: %s", entry.String(), maxConsumed.String()) 243 ckpData, err := c.collectCkpData(entry) 244 if err != nil { 245 logutil.Warnf("load checkpoint data failed, err[%v]", err) 246 continue 247 } 248 c.snapshotMeta.InitTableInfo(ckpData) 249 } 250 logutil.Infof("table info initialized: %s", c.snapshotMeta.TableInfoString()) 251 } 252 return nil 253 254 } 255 256 func (c *checkpointCleaner) updateMaxConsumed(e *checkpoint.CheckpointEntry) { 257 c.maxConsumed.Store(e) 258 } 259 260 func (c *checkpointCleaner) updateMinMerged(e *checkpoint.CheckpointEntry) { 261 c.minMerged.Store(e) 262 } 263 264 func (c *checkpointCleaner) updateMaxCompared(e *checkpoint.CheckpointEntry) { 265 c.maxCompared.Store(e) 266 } 267 268 func (c *checkpointCleaner) updateCkpStage(ts *types.TS) { 269 c.ckpStage.Store(ts) 270 } 271 272 func (c *checkpointCleaner) updateCkpGC(ts *types.TS) { 273 c.ckpGC.Store(ts) 274 } 275 276 func (c *checkpointCleaner) updateInputs(input *GCTable) { 277 c.inputs.Lock() 278 defer c.inputs.Unlock() 279 c.inputs.tables = append(c.inputs.tables, input) 280 } 281 282 func (c *checkpointCleaner) updateOutputs(files []string) { 283 c.outputs.Lock() 284 defer c.outputs.Unlock() 285 c.outputs.files = append(c.outputs.files, files...) 286 } 287 288 func (c *checkpointCleaner) GetMaxConsumed() *checkpoint.CheckpointEntry { 289 return c.maxConsumed.Load() 290 } 291 292 func (c *checkpointCleaner) GetMinMerged() *checkpoint.CheckpointEntry { 293 return c.minMerged.Load() 294 } 295 296 func (c *checkpointCleaner) GetMaxCompared() *checkpoint.CheckpointEntry { 297 return c.maxCompared.Load() 298 } 299 300 func (c *checkpointCleaner) GeteCkpStage() *types.TS { 301 return c.ckpStage.Load() 302 } 303 304 func (c *checkpointCleaner) GeteCkpGC() *types.TS { 305 return c.ckpGC.Load() 306 } 307 308 func (c *checkpointCleaner) GetInputs() *GCTable { 309 c.inputs.RLock() 310 defer c.inputs.RUnlock() 311 return c.inputs.tables[0] 312 } 313 314 func (c *checkpointCleaner) SetMinMergeCountForTest(count int) { 315 c.minMergeCount.Lock() 316 defer c.minMergeCount.Unlock() 317 c.minMergeCount.count = count 318 } 319 320 func (c *checkpointCleaner) getMinMergeCount() int { 321 c.minMergeCount.RLock() 322 defer c.minMergeCount.RUnlock() 323 return c.minMergeCount.count 324 } 325 326 func (c *checkpointCleaner) GetAndClearOutputs() []string { 327 c.outputs.RLock() 328 defer c.outputs.RUnlock() 329 files := c.outputs.files 330 //Empty the array, in order to store the next file list 331 c.outputs.files = make([]string, 0) 332 return files 333 } 334 335 func (c *checkpointCleaner) mergeGCFile() error { 336 maxConsumed := c.GetMaxConsumed() 337 if maxConsumed == nil { 338 return nil 339 } 340 maxSnapEnd := types.TS{} 341 maxAcctEnd := types.TS{} 342 var snapFile, acctFile string 343 dirs, err := c.fs.ListDir(GCMetaDir) 344 if err != nil { 345 return err 346 } 347 deleteFiles := make([]string, 0) 348 mergeSnapAcctFile := func(name string, ts, max *types.TS, file *string) error { 349 if *file != "" { 350 if max.Less(ts) { 351 max = ts 352 err = c.fs.Delete(*file) 353 if err != nil { 354 logutil.Errorf("DelFiles failed: %v, max: %v", err.Error(), max.ToString()) 355 return err 356 } 357 *file = GCMetaDir + name 358 } else { 359 err = c.fs.Delete(GCMetaDir + name) 360 if err != nil { 361 logutil.Errorf("DelFiles failed: %v, max: %v", err.Error(), max.ToString()) 362 return err 363 } 364 } 365 } else { 366 *file = GCMetaDir + name 367 max = ts 368 logutil.Infof("mergeSnapAcctFile: %v, max: %v", name, max.ToString()) 369 } 370 return nil 371 } 372 for _, dir := range dirs { 373 _, ts, ext := blockio.DecodeGCMetadataFileName(dir.Name) 374 if ext == blockio.SnapshotExt { 375 err = mergeSnapAcctFile(dir.Name, &ts, &maxSnapEnd, &snapFile) 376 if err != nil { 377 return err 378 } 379 continue 380 } 381 if ext == blockio.AcctExt { 382 err = mergeSnapAcctFile(dir.Name, &ts, &maxAcctEnd, &acctFile) 383 if err != nil { 384 return err 385 } 386 continue 387 } 388 _, end := blockio.DecodeCheckpointMetadataFileName(dir.Name) 389 maxEnd := maxConsumed.GetEnd() 390 if end.LessEq(&maxEnd) { 391 deleteFiles = append(deleteFiles, GCMetaDir+dir.Name) 392 } 393 } 394 if len(deleteFiles) < c.getMinMergeCount() { 395 return nil 396 } 397 var mergeTable *GCTable 398 c.inputs.RLock() 399 if len(c.inputs.tables) == 0 { 400 c.inputs.RUnlock() 401 return nil 402 } 403 // tables[0] has always been a full GCTable 404 if len(c.inputs.tables) > 1 { 405 mergeTable = NewGCTable() 406 for _, table := range c.inputs.tables { 407 mergeTable.Merge(table) 408 } 409 } else { 410 mergeTable = c.inputs.tables[0] 411 } 412 c.inputs.RUnlock() 413 logutil.Info("[DiskCleaner]", 414 common.OperationField("MergeGCFile start"), 415 common.OperandField(maxConsumed.String())) 416 _, err = mergeTable.SaveFullTable(maxConsumed.GetStart(), maxConsumed.GetEnd(), c.fs, nil) 417 if err != nil { 418 logutil.Errorf("SaveTable failed: %v", err.Error()) 419 return err 420 } 421 err = c.fs.DelFiles(c.ctx, deleteFiles) 422 if err != nil { 423 logutil.Errorf("DelFiles failed: %v", err.Error()) 424 return err 425 } 426 c.updateMinMerged(maxConsumed) 427 logutil.Info("[DiskCleaner]", 428 common.OperationField("MergeGCFile end"), 429 common.OperandField(maxConsumed.String())) 430 return nil 431 } 432 433 func (c *checkpointCleaner) mergeCheckpointFiles(stage types.TS) error { 434 if stage.IsEmpty() || 435 (c.GeteCkpStage() != nil && c.GeteCkpStage().GreaterEq(&stage)) { 436 return nil 437 } 438 logutil.Infof("mergeCheckpointFiles stage: %v", stage.ToString()) 439 files, idx, err := checkpoint.ListSnapshotMeta(c.ctx, c.fs.Service, stage, nil) 440 if err != nil { 441 return err 442 } 443 if len(files) == 0 { 444 return nil 445 } 446 ckpGC := c.GeteCkpGC() 447 if ckpGC == nil { 448 ckpGC = new(types.TS) 449 } 450 ckps, err := checkpoint.ListSnapshotCheckpointWithMeta(c.ctx, c.fs.Service, files, idx, *ckpGC, true) 451 if err != nil { 452 return err 453 } 454 if len(ckps) == 0 { 455 return nil 456 } 457 deleteFiles := make([]string, 0) 458 for _, ckp := range ckps { 459 end := ckp.GetEnd() 460 if end.Less(&stage) { 461 logutil.Infof("GC checkpoint: %v, %v", ckp.GetStart().ToString(), end.ToString()) 462 locations, err := logtail.LoadCheckpointLocations(c.ctx, ckp.GetTNLocation(), ckp.GetVersion(), c.fs.Service) 463 if err != nil { 464 if moerr.IsMoErrCode(err, moerr.ErrFileNotFound) { 465 continue 466 } 467 return err 468 } 469 for name := range locations { 470 deleteFiles = append(deleteFiles, name) 471 } 472 deleteFiles = append(deleteFiles, ckp.GetTNLocation().Name().String()) 473 } 474 475 } 476 for i := 0; i < idx+1; i++ { 477 end := files[i].GetEnd() 478 if end.Less(&stage) { 479 deleteFiles = append(deleteFiles, CKPMetaDir+files[i].GetName()) 480 } 481 } 482 logutil.Infof("CKP GC: %v", deleteFiles) 483 if !c.disableGC { 484 err = c.fs.DelFiles(c.ctx, deleteFiles) 485 if err != nil { 486 logutil.Errorf("DelFiles failed: %v", err.Error()) 487 return err 488 } 489 } 490 c.updateCkpStage(&stage) 491 c.updateCkpGC(&stage) 492 return nil 493 } 494 495 func (c *checkpointCleaner) collectGlobalCkpData( 496 ckp *checkpoint.CheckpointEntry, 497 ) (data *logtail.CheckpointData, err error) { 498 _, data, err = logtail.LoadCheckpointEntriesFromKey(c.ctx, c.fs.Service, 499 ckp.GetLocation(), ckp.GetVersion(), nil, &types.TS{}) 500 return 501 } 502 503 func (c *checkpointCleaner) collectCkpData( 504 ckp *checkpoint.CheckpointEntry, 505 ) (data *logtail.CheckpointData, err error) { 506 _, data, err = logtail.LoadCheckpointEntriesFromKey(c.ctx, c.fs.Service, 507 ckp.GetLocation(), ckp.GetVersion(), nil, &types.TS{}) 508 return 509 } 510 511 func (c *checkpointCleaner) TryGC() error { 512 maxGlobalCKP := c.ckpClient.MaxGlobalCheckpoint() 513 if maxGlobalCKP != nil { 514 data, err := c.collectGlobalCkpData(maxGlobalCKP) 515 if err != nil { 516 return err 517 } 518 defer data.Close() 519 err = c.tryGC(data, maxGlobalCKP) 520 if err != nil { 521 return err 522 } 523 } 524 return nil 525 } 526 527 func (c *checkpointCleaner) tryGC(data *logtail.CheckpointData, gckp *checkpoint.CheckpointEntry) error { 528 if !c.delWorker.Start() { 529 return nil 530 } 531 gcTable := NewGCTable() 532 gcTable.UpdateTable(data) 533 snapshots, err := c.GetSnapshots() 534 if err != nil { 535 logutil.Errorf("[DiskCleaner] GetSnapshots failed: %v", err.Error()) 536 return nil 537 } 538 defer logtail.CloseSnapshotList(snapshots) 539 gc := c.softGC(gcTable, gckp, snapshots) 540 // Delete files after softGC 541 // TODO:Requires Physical Removal Policy 542 err = c.delWorker.ExecDelete(c.ctx, gc, c.disableGC) 543 if err != nil { 544 return err 545 } 546 return nil 547 } 548 549 func (c *checkpointCleaner) softGC(t *GCTable, gckp *checkpoint.CheckpointEntry, snapshots map[uint32]containers.Vector) []string { 550 c.inputs.Lock() 551 defer c.inputs.Unlock() 552 if len(c.inputs.tables) == 0 { 553 return nil 554 } 555 mergeTable := NewGCTable() 556 for _, table := range c.inputs.tables { 557 mergeTable.Merge(table) 558 } 559 gc, snapList := mergeTable.SoftGC(t, gckp.GetEnd(), snapshots, c.snapshotMeta) 560 c.inputs.tables = make([]*GCTable, 0) 561 c.inputs.tables = append(c.inputs.tables, mergeTable) 562 c.updateMaxCompared(gckp) 563 c.snapshotMeta.MergeTableInfo(snapList) 564 //logutil.Infof("SoftGC is %v, merge table: %v", gc, mergeTable.String()) 565 return gc 566 } 567 568 func (c *checkpointCleaner) createDebugInput( 569 ckps []*checkpoint.CheckpointEntry) (input *GCTable, err error) { 570 input = NewGCTable() 571 var data *logtail.CheckpointData 572 for _, candidate := range ckps { 573 data, err = c.collectCkpData(candidate) 574 if err != nil { 575 logutil.Errorf("processing clean %s: %v", candidate.String(), err) 576 // TODO 577 return 578 } 579 defer data.Close() 580 input.UpdateTable(data) 581 } 582 583 return 584 } 585 586 func (c *checkpointCleaner) CheckGC() error { 587 debugCandidates := c.ckpClient.GetAllIncrementalCheckpoints() 588 c.inputs.RLock() 589 defer c.inputs.RUnlock() 590 maxConsumed := c.GetMaxConsumed() 591 if maxConsumed == nil { 592 return moerr.NewInternalErrorNoCtx("GC has not yet run") 593 } 594 gCkp := c.GetMaxCompared() 595 if gCkp == nil { 596 gCkp = c.ckpClient.MaxGlobalCheckpoint() 597 if gCkp == nil { 598 return nil 599 } 600 logutil.Warnf("MaxCompared is nil, use maxGlobalCkp %v", gCkp.String()) 601 } 602 data, err := c.collectGlobalCkpData(gCkp) 603 if err != nil { 604 return err 605 } 606 defer data.Close() 607 gcTable := NewGCTable() 608 gcTable.UpdateTable(data) 609 for i, ckp := range debugCandidates { 610 maxEnd := maxConsumed.GetEnd() 611 ckpEnd := ckp.GetEnd() 612 if ckpEnd.Equal(&maxEnd) { 613 debugCandidates = debugCandidates[:i+1] 614 break 615 } 616 } 617 start1 := debugCandidates[len(debugCandidates)-1].GetEnd() 618 start2 := maxConsumed.GetEnd() 619 if !start1.Equal(&start2) { 620 logutil.Info("[DiskCleaner]", common.OperationField("Compare not equal"), 621 common.OperandField(start1.ToString()), common.OperandField(start2.ToString())) 622 return moerr.NewInternalErrorNoCtx("TS Compare not equal") 623 } 624 debugTable, err := c.createDebugInput(debugCandidates) 625 if err != nil { 626 logutil.Errorf("processing clean %s: %v", debugCandidates[0].String(), err) 627 // TODO 628 return moerr.NewInternalErrorNoCtx("processing clean %s: %v", debugCandidates[0].String(), err) 629 } 630 snapshots, err := c.GetSnapshots() 631 if err != nil { 632 logutil.Errorf("processing clean %s: %v", debugCandidates[0].String(), err) 633 return moerr.NewInternalErrorNoCtx("processing clean GetSnapshots %s: %v", debugCandidates[0].String(), err) 634 } 635 defer logtail.CloseSnapshotList(snapshots) 636 debugTable.SoftGC(gcTable, gCkp.GetEnd(), snapshots, c.snapshotMeta) 637 var mergeTable *GCTable 638 if len(c.inputs.tables) > 1 { 639 mergeTable = NewGCTable() 640 for _, table := range c.inputs.tables { 641 mergeTable.Merge(table) 642 } 643 } else { 644 mergeTable = c.inputs.tables[0] 645 } 646 mergeTable.SoftGC(gcTable, gCkp.GetEnd(), snapshots, c.snapshotMeta) 647 if !mergeTable.Compare(debugTable) { 648 logutil.Errorf("inputs :%v", c.inputs.tables[0].String()) 649 logutil.Errorf("debugTable :%v", debugTable.String()) 650 return moerr.NewInternalErrorNoCtx("Compare is failed") 651 } else { 652 logutil.Info("[DiskCleaner]", common.OperationField("Compare is End"), 653 common.AnyField("table :", debugTable.String()), 654 common.OperandField(start1.ToString())) 655 } 656 return nil 657 } 658 659 func (c *checkpointCleaner) Process() { 660 var ts types.TS 661 if !c.isEnableGC() { 662 return 663 } 664 665 maxConsumed := c.maxConsumed.Load() 666 if maxConsumed != nil { 667 ts = maxConsumed.GetEnd() 668 } 669 670 checkpoints := c.ckpClient.ICKPSeekLT(ts, 10) 671 672 if len(checkpoints) == 0 { 673 return 674 } 675 candidates := make([]*checkpoint.CheckpointEntry, 0) 676 for _, ckp := range checkpoints { 677 if !c.checkExtras(ckp) { 678 break 679 } 680 candidates = append(candidates, ckp) 681 } 682 683 if len(candidates) == 0 { 684 return 685 } 686 var input *GCTable 687 var err error 688 if input, err = c.createNewInput(candidates); err != nil { 689 logutil.Errorf("[DiskCleaner] processing clean %s: %v", candidates[0].String(), err) 690 // TODO 691 return 692 } 693 c.updateInputs(input) 694 c.updateMaxConsumed(candidates[len(candidates)-1]) 695 696 var compareTS types.TS 697 maxCompared := c.maxCompared.Load() 698 if maxCompared != nil { 699 compareTS = maxCompared.GetEnd() 700 } 701 maxGlobalCKP := c.ckpClient.MaxGlobalCheckpoint() 702 if maxGlobalCKP == nil { 703 return 704 } 705 maxEnd := maxGlobalCKP.GetEnd() 706 if maxGlobalCKP != nil && compareTS.Less(&maxEnd) { 707 logutil.Info("[DiskCleaner]", common.OperationField("Try GC"), 708 common.AnyField("maxGlobalCKP :", maxGlobalCKP.String()), 709 common.AnyField("compareTS :", compareTS.ToString())) 710 data, err := c.collectGlobalCkpData(maxGlobalCKP) 711 if err != nil { 712 c.inputs.RUnlock() 713 logutil.Errorf("[DiskCleaner] processing clean %s: %v", candidates[0].String(), err) 714 return 715 } 716 defer data.Close() 717 err = c.tryGC(data, maxGlobalCKP) 718 if err != nil { 719 logutil.Errorf("[DiskCleaner] processing clean %s: %v", candidates[0].String(), err) 720 return 721 } 722 } 723 err = c.mergeGCFile() 724 if err != nil { 725 // TODO: Error handle 726 return 727 } 728 729 err = c.mergeCheckpointFiles(c.ckpClient.GetStage()) 730 731 if err != nil { 732 // TODO: Error handle 733 return 734 } 735 } 736 737 func (c *checkpointCleaner) checkExtras(item any) bool { 738 c.checker.RLock() 739 defer c.checker.RUnlock() 740 for _, checker := range c.checker.extras { 741 if !checker(item) { 742 return false 743 } 744 } 745 return true 746 } 747 748 func (c *checkpointCleaner) AddChecker(checker func(item any) bool) { 749 c.checker.Lock() 750 defer c.checker.Unlock() 751 c.checker.extras = append(c.checker.extras, checker) 752 } 753 754 func (c *checkpointCleaner) createNewInput( 755 ckps []*checkpoint.CheckpointEntry) (input *GCTable, err error) { 756 now := time.Now() 757 var snapSize, tableSize uint32 758 logutil.Info("[DiskCleaner]", common.OperationField("Consume-Start"), 759 common.AnyField("entry count :", len(ckps))) 760 defer func() { 761 logutil.Info("[DiskCleaner]", common.OperationField("Consume-End"), 762 common.AnyField("cost :", time.Since(now).String()), 763 common.AnyField("snap meta size :", snapSize), 764 common.AnyField("table meta size :", tableSize), 765 common.OperandField(c.snapshotMeta.String())) 766 }() 767 input = NewGCTable() 768 var data *logtail.CheckpointData 769 for _, candidate := range ckps { 770 data, err = c.collectCkpData(candidate) 771 if err != nil { 772 logutil.Errorf("processing clean %s: %v", candidate.String(), err) 773 // TODO 774 return 775 } 776 defer data.Close() 777 input.UpdateTable(data) 778 c.updateSnapshot(data) 779 } 780 name := blockio.EncodeSnapshotMetadataFileName(GCMetaDir, 781 PrefixSnapMeta, ckps[0].GetStart(), ckps[len(ckps)-1].GetEnd()) 782 snapSize, err = c.snapshotMeta.SaveMeta(name, c.fs.Service) 783 if err != nil { 784 logutil.Errorf("SaveMeta is failed") 785 return 786 } 787 name = blockio.EncodeTableMetadataFileName(GCMetaDir, 788 PrefixAcctMeta, ckps[0].GetStart(), ckps[len(ckps)-1].GetEnd()) 789 tableSize, err = c.snapshotMeta.SaveTableInfo(name, c.fs.Service) 790 if err != nil { 791 logutil.Errorf("SaveTableInfo is failed") 792 return 793 } 794 files := c.GetAndClearOutputs() 795 _, err = input.SaveTable( 796 ckps[0].GetStart(), 797 ckps[len(ckps)-1].GetEnd(), 798 c.fs, 799 files, 800 ) 801 if err != nil { 802 return 803 } 804 805 return 806 } 807 808 func (c *checkpointCleaner) updateSnapshot(data *logtail.CheckpointData) error { 809 c.snapshotMeta.Update(data) 810 return nil 811 } 812 813 func (c *checkpointCleaner) GetSnapshots() (map[uint32]containers.Vector, error) { 814 return c.snapshotMeta.GetSnapshot(c.ctx, c.fs.Service, c.mPool) 815 }