github.com/matrixorigin/matrixone@v0.7.0/pkg/vm/engine/tae/db/gc/diskcleaner.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 "fmt" 20 "github.com/matrixorigin/matrixone/pkg/common/moerr" 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/catalog" 26 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/common" 27 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/dataio/blockio" 28 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/db/checkpoint" 29 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logstore/sm" 30 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/logtail" 31 "github.com/matrixorigin/matrixone/pkg/vm/engine/tae/tasks" 32 "sync" 33 "sync/atomic" 34 ) 35 36 const ( 37 MessgeReplay = iota 38 MessgeNormal 39 ) 40 41 const MinMergeCount = 20 42 43 // DiskCleaner is the main structure of gc operation, 44 // and provides "JobFactory" to let tae notify itself 45 // to perform a gc 46 type DiskCleaner struct { 47 fs *objectio.ObjectFS 48 49 // ckpClient is used to get the instance of the specified checkpoint 50 ckpClient checkpoint.RunnerReader 51 52 // Parsing checkpoint needs to use catalog instance 53 catalog *catalog.Catalog 54 55 // maxConsumed is to mark which checkpoint the current DiskCleaner has processed, 56 // through which you can get the next checkpoint to be processed 57 maxConsumed atomic.Pointer[checkpoint.CheckpointEntry] 58 59 // minMerged is to mark at which checkpoint the full 60 // GCTable in the current DiskCleaner is generated, 61 // UT case needs to use 62 minMerged atomic.Pointer[checkpoint.CheckpointEntry] 63 64 // minMergeCount is the configuration of the merge GC metadata file. 65 // When the GC file is greater than or equal to minMergeCount, 66 // the merge GC metadata file will be triggered and the expired file will be deleted. 67 minMergeCount struct { 68 sync.RWMutex 69 count int 70 } 71 72 // inputs is to record the currently valid GCTable 73 inputs struct { 74 sync.RWMutex 75 tables []*GCTable 76 } 77 78 // outputs is a list of files that have been deleted 79 outputs struct { 80 sync.RWMutex 81 files []string 82 } 83 // delWorker is a worker that deletes s3‘s objects or local 84 // files, and only one worker will run 85 delWorker *GCWorker 86 87 // checker is to check whether the checkpoint can be consumed 88 checker struct { 89 sync.RWMutex 90 extras []func(item any) bool 91 } 92 93 processQueue sm.Queue 94 95 onceStart sync.Once 96 onceStop sync.Once 97 } 98 99 func NewDiskCleaner( 100 fs *objectio.ObjectFS, 101 ckpClient checkpoint.RunnerReader, 102 catalog *catalog.Catalog, 103 ) *DiskCleaner { 104 cleaner := &DiskCleaner{ 105 fs: fs, 106 ckpClient: ckpClient, 107 catalog: catalog, 108 } 109 cleaner.delWorker = NewGCWorker(fs, cleaner) 110 cleaner.processQueue = sm.NewSafeQueue(10000, 1000, cleaner.process) 111 cleaner.minMergeCount.count = MinMergeCount 112 return cleaner 113 } 114 115 func (cleaner *DiskCleaner) GC(ctx context.Context) (err error) { 116 logutil.Info("JobFactory is start") 117 return cleaner.tryClean(ctx) 118 } 119 120 func (cleaner *DiskCleaner) tryReplay() { 121 if _, err := cleaner.processQueue.Enqueue(MessgeReplay); err != nil { 122 panic(err) 123 } 124 } 125 126 func (cleaner *DiskCleaner) tryClean(ctx context.Context) (err error) { 127 _, err = cleaner.processQueue.Enqueue(MessgeNormal) 128 return 129 } 130 131 func (cleaner *DiskCleaner) replay() error { 132 dirs, err := cleaner.fs.ListDir(GCMetaDir) 133 if err != nil { 134 return err 135 } 136 if len(dirs) == 0 { 137 return nil 138 } 139 minMergedStart := types.TS{} 140 minMergedEnd := types.TS{} 141 maxConsumedStart := types.TS{} 142 maxConsumedEnd := types.TS{} 143 var fullGCFile fileservice.DirEntry 144 // Get effective minMerged 145 for _, dir := range dirs { 146 start, end, ext := blockio.DecodeGCMetadataFileName(dir.Name) 147 if ext == blockio.GCFullExt { 148 if minMergedStart.IsEmpty() || minMergedStart.Less(start) { 149 minMergedStart = start 150 minMergedEnd = end 151 maxConsumedStart = start 152 maxConsumedEnd = end 153 fullGCFile = dir 154 } 155 } 156 } 157 readDirs := make([]fileservice.DirEntry, 0) 158 if !minMergedStart.IsEmpty() { 159 readDirs = append(readDirs, fullGCFile) 160 } 161 logutil.Infof("minMergedEnd is %v", minMergedEnd.ToString()) 162 for _, dir := range dirs { 163 start, end, ext := blockio.DecodeGCMetadataFileName(dir.Name) 164 if ext == blockio.GCFullExt { 165 continue 166 } 167 if (maxConsumedStart.IsEmpty() || maxConsumedStart.Less(end)) && 168 minMergedEnd.Less(end) { 169 maxConsumedStart = start 170 maxConsumedEnd = end 171 readDirs = append(readDirs, dir) 172 } 173 } 174 if len(readDirs) == 0 { 175 return nil 176 } 177 jobs := make([]*tasks.Job, len(readDirs)) 178 jobScheduler := tasks.NewParallelJobScheduler(100) 179 defer jobScheduler.Stop() 180 makeJob := func(i int) (job *tasks.Job) { 181 dir := readDirs[i] 182 exec := func(ctx context.Context) (result *tasks.JobResult) { 183 result = &tasks.JobResult{} 184 table := NewGCTable() 185 err := table.ReadTable(ctx, GCMetaDir+dir.Name, dir.Size, cleaner.fs) 186 if err != nil { 187 result.Err = err 188 return 189 } 190 cleaner.updateInputs(table) 191 return 192 } 193 job = tasks.NewJob( 194 fmt.Sprintf("load-%s", dir.Name), 195 context.Background(), 196 exec) 197 return 198 } 199 200 for i := range readDirs { 201 jobs[i] = makeJob(i) 202 if err = jobScheduler.Schedule(jobs[i]); err != nil { 203 return err 204 } 205 } 206 207 for _, job := range jobs { 208 result := job.WaitDone() 209 if err = result.Err; err != nil { 210 return err 211 } 212 } 213 ckp := checkpoint.NewCheckpointEntry(maxConsumedStart, maxConsumedEnd, checkpoint.ET_Incremental) 214 cleaner.updateMaxConsumed(ckp) 215 ckp = checkpoint.NewCheckpointEntry(minMergedStart, minMergedEnd, checkpoint.ET_Incremental) 216 cleaner.updateMinMerged(ckp) 217 return nil 218 } 219 220 func (cleaner *DiskCleaner) process(items ...any) { 221 if items[0].(int) == MessgeReplay { 222 err := cleaner.replay() 223 if err != nil { 224 panic(err) 225 } 226 // TODO: 227 cleaner.tryGC() 228 if len(items) == 1 { 229 return 230 } 231 } 232 233 var ts types.TS 234 maxConsumed := cleaner.maxConsumed.Load() 235 if maxConsumed != nil { 236 ts = maxConsumed.GetEnd() 237 } 238 239 checkpoints := cleaner.ckpClient.ICKPSeekLT(ts, 10) 240 241 if len(checkpoints) == 0 { 242 return 243 } 244 candidates := make([]*checkpoint.CheckpointEntry, 0) 245 for _, ckp := range checkpoints { 246 if !cleaner.checkExtras(ckp) { 247 break 248 } 249 candidates = append(candidates, ckp) 250 } 251 252 if len(candidates) == 0 { 253 return 254 } 255 var input *GCTable 256 var err error 257 if input, err = cleaner.createNewInput(candidates); err != nil { 258 logutil.Errorf("processing clean %s: %v", candidates[0].String(), err) 259 // TODO 260 return 261 } 262 cleaner.updateInputs(input) 263 cleaner.updateMaxConsumed(candidates[len(candidates)-1]) 264 err = cleaner.tryGC() 265 if err != nil { 266 return 267 } 268 err = cleaner.mergeGCFile() 269 if err != nil { 270 // TODO: Error handle 271 return 272 } 273 } 274 275 func (cleaner *DiskCleaner) checkExtras(item any) bool { 276 cleaner.checker.RLock() 277 defer cleaner.checker.RUnlock() 278 for _, checker := range cleaner.checker.extras { 279 if !checker(item) { 280 return false 281 } 282 } 283 return true 284 } 285 286 func (cleaner *DiskCleaner) AddChecker(checker func(item any) bool) { 287 cleaner.checker.Lock() 288 defer cleaner.checker.Unlock() 289 cleaner.checker.extras = append(cleaner.checker.extras, checker) 290 } 291 292 func (cleaner *DiskCleaner) SetMinMergeCountForTest(count int) { 293 cleaner.minMergeCount.Lock() 294 defer cleaner.minMergeCount.Unlock() 295 cleaner.minMergeCount.count = count 296 } 297 298 func (cleaner *DiskCleaner) getMinMergeCount() int { 299 cleaner.minMergeCount.RLock() 300 defer cleaner.minMergeCount.RUnlock() 301 return cleaner.minMergeCount.count 302 } 303 304 func (cleaner *DiskCleaner) collectCkpData( 305 ckp *checkpoint.CheckpointEntry, 306 ) (data *logtail.CheckpointData, err error) { 307 factory := logtail.IncrementalCheckpointDataFactory( 308 ckp.GetStart(), 309 ckp.GetEnd(), 310 ) 311 data, err = factory(cleaner.catalog) 312 return 313 } 314 315 func (cleaner *DiskCleaner) createNewInput( 316 ckps []*checkpoint.CheckpointEntry) (input *GCTable, err error) { 317 input = NewGCTable() 318 var data *logtail.CheckpointData 319 for _, candidate := range ckps { 320 data, err = cleaner.collectCkpData(candidate) 321 if err != nil { 322 logutil.Errorf("processing clean %s: %v", candidate.String(), err) 323 // TODO 324 return 325 } 326 defer data.Close() 327 input.UpdateTable(data) 328 } 329 files := cleaner.GetAndClearOutputs() 330 _, err = input.SaveTable( 331 ckps[0].GetStart(), 332 ckps[len(ckps)-1].GetEnd(), 333 cleaner.fs, 334 files, 335 ) 336 if err != nil { 337 return 338 } 339 340 return 341 } 342 343 func (cleaner *DiskCleaner) createDebugInput( 344 ckps []*checkpoint.CheckpointEntry) (input *GCTable, err error) { 345 input = NewGCTable() 346 var data *logtail.CheckpointData 347 for _, candidate := range ckps { 348 data, err = cleaner.collectCkpData(candidate) 349 if err != nil { 350 logutil.Errorf("processing clean %s: %v", candidate.String(), err) 351 // TODO 352 return 353 } 354 defer data.Close() 355 input.UpdateTable(data) 356 } 357 358 return 359 } 360 361 func (cleaner *DiskCleaner) tryGC() error { 362 if !cleaner.delWorker.Start() { 363 return nil 364 } 365 gc := cleaner.softGC() 366 // Delete files after softGC 367 // TODO:Requires Physical Removal Policy 368 err := cleaner.delWorker.ExecDelete(gc) 369 if err != nil { 370 return err 371 } 372 return nil 373 } 374 375 func (cleaner *DiskCleaner) softGC() []string { 376 cleaner.inputs.Lock() 377 defer cleaner.inputs.Unlock() 378 if len(cleaner.inputs.tables) == 0 { 379 return nil 380 } 381 mergeTable := NewGCTable() 382 for _, table := range cleaner.inputs.tables { 383 mergeTable.Merge(table) 384 } 385 gc := mergeTable.SoftGC() 386 cleaner.inputs.tables = make([]*GCTable, 0) 387 cleaner.inputs.tables = append(cleaner.inputs.tables, mergeTable) 388 //logutil.Infof("SoftGC is %v, merge table: %v", gc, mergeTable.String()) 389 return gc 390 } 391 392 func (cleaner *DiskCleaner) updateMaxConsumed(e *checkpoint.CheckpointEntry) { 393 cleaner.maxConsumed.Store(e) 394 } 395 396 func (cleaner *DiskCleaner) updateMinMerged(e *checkpoint.CheckpointEntry) { 397 cleaner.minMerged.Store(e) 398 } 399 400 func (cleaner *DiskCleaner) updateInputs(input *GCTable) { 401 cleaner.inputs.Lock() 402 defer cleaner.inputs.Unlock() 403 cleaner.inputs.tables = append(cleaner.inputs.tables, input) 404 } 405 406 func (cleaner *DiskCleaner) updateOutputs(files []string) { 407 cleaner.outputs.Lock() 408 defer cleaner.outputs.Unlock() 409 cleaner.outputs.files = append(cleaner.outputs.files, files...) 410 } 411 412 func (cleaner *DiskCleaner) GetMaxConsumed() *checkpoint.CheckpointEntry { 413 return cleaner.maxConsumed.Load() 414 } 415 416 func (cleaner *DiskCleaner) GetMinMerged() *checkpoint.CheckpointEntry { 417 return cleaner.minMerged.Load() 418 } 419 420 func (cleaner *DiskCleaner) GetInputs() *GCTable { 421 cleaner.inputs.RLock() 422 defer cleaner.inputs.RUnlock() 423 return cleaner.inputs.tables[0] 424 } 425 426 func (cleaner *DiskCleaner) GetAndClearOutputs() []string { 427 cleaner.outputs.RLock() 428 defer cleaner.outputs.RUnlock() 429 files := cleaner.outputs.files 430 //Empty the array, in order to store the next file list 431 cleaner.outputs.files = make([]string, 0) 432 return files 433 } 434 435 func (cleaner *DiskCleaner) mergeGCFile() error { 436 maxConsumed := cleaner.GetMaxConsumed() 437 if maxConsumed == nil { 438 return nil 439 } 440 dirs, err := cleaner.fs.ListDir(GCMetaDir) 441 if err != nil { 442 return err 443 } 444 deleteFiles := make([]string, 0) 445 for _, dir := range dirs { 446 _, end := blockio.DecodeCheckpointMetadataFileName(dir.Name) 447 if end.LessEq(maxConsumed.GetEnd()) { 448 deleteFiles = append(deleteFiles, GCMetaDir+dir.Name) 449 } 450 } 451 if len(deleteFiles) < cleaner.getMinMergeCount() { 452 return nil 453 } 454 455 var mergeTable *GCTable 456 cleaner.inputs.RLock() 457 if len(cleaner.inputs.tables) == 0 { 458 cleaner.inputs.RUnlock() 459 return nil 460 } 461 // tables[0] has always been a full GCTable 462 mergeTable = cleaner.inputs.tables[0] 463 cleaner.inputs.RUnlock() 464 logutil.Info("[DiskCleaner]", 465 common.OperationField("MergeGCFile start"), 466 common.OperandField(maxConsumed.String())) 467 _, err = mergeTable.SaveFullTable(maxConsumed.GetStart(), maxConsumed.GetEnd(), cleaner.fs, nil) 468 if err != nil { 469 logutil.Errorf("SaveTable failed: %v", err.Error()) 470 return err 471 } 472 err = cleaner.fs.DelFiles(context.Background(), deleteFiles) 473 if err != nil { 474 logutil.Errorf("DelFiles failed: %v", err.Error()) 475 return err 476 } 477 cleaner.updateMinMerged(maxConsumed) 478 logutil.Info("[DiskCleaner]", 479 common.OperationField("MergeGCFile end"), 480 common.OperandField(maxConsumed.String())) 481 return nil 482 } 483 484 func (cleaner *DiskCleaner) CheckGC() error { 485 debugCandidates := cleaner.ckpClient.GetAllIncrementalCheckpoints() 486 cleaner.inputs.RLock() 487 defer cleaner.inputs.RUnlock() 488 maxConsumed := cleaner.GetMaxConsumed() 489 if maxConsumed == nil { 490 return moerr.NewInternalErrorNoCtx("GC has not yet run") 491 } 492 for i, ckp := range debugCandidates { 493 if ckp.GetEnd().Equal(maxConsumed.GetEnd()) { 494 debugCandidates = debugCandidates[:i+1] 495 break 496 } 497 } 498 start1 := debugCandidates[len(debugCandidates)-1].GetEnd() 499 start2 := maxConsumed.GetEnd() 500 if !start1.Equal(start2) { 501 logutil.Info("[DiskCleaner]", common.OperationField("Compare not equal"), 502 common.OperandField(start1.ToString()), common.OperandField(start2.ToString())) 503 return moerr.NewInternalErrorNoCtx("TS Compare not equal") 504 } 505 debugTable, err := cleaner.createDebugInput(debugCandidates) 506 if err != nil { 507 logutil.Errorf("processing clean %s: %v", debugCandidates[0].String(), err) 508 // TODO 509 return moerr.NewInternalErrorNoCtx("processing clean %s: %v", debugCandidates[0].String(), err) 510 } 511 debugTable.SoftGC() 512 var mergeTable *GCTable 513 if len(cleaner.inputs.tables) > 1 { 514 mergeTable = NewGCTable() 515 for _, table := range cleaner.inputs.tables { 516 mergeTable.Merge(table) 517 } 518 mergeTable.SoftGC() 519 } else { 520 mergeTable = cleaner.inputs.tables[0] 521 } 522 if !mergeTable.Compare(debugTable) { 523 logutil.Errorf("inputs :%v", cleaner.inputs.tables[0].String()) 524 logutil.Errorf("debugTable :%v", debugTable.String()) 525 return moerr.NewInternalErrorNoCtx("Compare is failed") 526 } else { 527 logutil.Info("[DiskCleaner]", common.OperationField("Compare is End"), 528 common.OperandField(start1.ToString())) 529 } 530 return nil 531 } 532 533 func (cleaner *DiskCleaner) Start() { 534 cleaner.onceStart.Do(func() { 535 cleaner.processQueue.Start() 536 cleaner.tryReplay() 537 }) 538 } 539 540 func (cleaner *DiskCleaner) Stop() { 541 cleaner.onceStop.Do(func() { 542 cleaner.processQueue.Stop() 543 }) 544 }