github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/redo/meta_manager.go (about) 1 // Copyright 2023 PingCAP, Inc. 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 // See the License for the specific language governing permissions and 12 // limitations under the License. 13 14 package redo 15 16 import ( 17 "context" 18 "fmt" 19 "path/filepath" 20 "strings" 21 "time" 22 23 "github.com/pingcap/log" 24 "github.com/pingcap/tidb/br/pkg/storage" 25 "github.com/pingcap/tiflow/cdc/model" 26 "github.com/pingcap/tiflow/cdc/redo/common" 27 "github.com/pingcap/tiflow/pkg/config" 28 "github.com/pingcap/tiflow/pkg/errors" 29 "github.com/pingcap/tiflow/pkg/redo" 30 "github.com/pingcap/tiflow/pkg/util" 31 "github.com/pingcap/tiflow/pkg/uuid" 32 "github.com/prometheus/client_golang/prometheus" 33 "go.uber.org/atomic" 34 "go.uber.org/zap" 35 "golang.org/x/sync/errgroup" 36 ) 37 38 var _ MetaManager = (*metaManager)(nil) 39 40 // MetaManager defines an interface that is used to manage redo meta and gc logs in owner. 41 type MetaManager interface { 42 redoManager 43 // UpdateMeta updates the checkpointTs and resolvedTs asynchronously. 44 UpdateMeta(checkpointTs, resolvedTs model.Ts) 45 // GetFlushedMeta returns the flushed meta. 46 GetFlushedMeta() common.LogMeta 47 // Cleanup deletes all redo logs, which are only called from the owner 48 // when changefeed is deleted. 49 Cleanup(ctx context.Context) error 50 51 // Running return true if the meta manager is running or not. 52 Running() bool 53 } 54 55 type metaManager struct { 56 captureID model.CaptureID 57 changeFeedID model.ChangeFeedID 58 enabled bool 59 60 // running means the meta manager now running normally. 61 running atomic.Bool 62 63 metaCheckpointTs statefulRts 64 metaResolvedTs statefulRts 65 66 // This fields are used to process meta files and perform 67 // garbage collection of logs. 68 extStorage storage.ExternalStorage 69 uuidGenerator uuid.Generator 70 preMetaFile string 71 72 startTs model.Ts 73 74 lastFlushTime time.Time 75 cfg *config.ConsistentConfig 76 metricFlushLogDuration prometheus.Observer 77 78 flushIntervalInMs int64 79 } 80 81 // NewDisabledMetaManager creates a disabled Meta Manager. 82 func NewDisabledMetaManager() *metaManager { 83 return &metaManager{ 84 enabled: false, 85 } 86 } 87 88 // NewMetaManager creates a new meta Manager. 89 func NewMetaManager( 90 changefeedID model.ChangeFeedID, cfg *config.ConsistentConfig, checkpoint model.Ts, 91 ) *metaManager { 92 // return a disabled Manager if no consistent config or normal consistent level 93 if cfg == nil || !redo.IsConsistentEnabled(cfg.Level) { 94 return &metaManager{enabled: false} 95 } 96 97 m := &metaManager{ 98 captureID: config.GetGlobalServerConfig().AdvertiseAddr, 99 changeFeedID: changefeedID, 100 uuidGenerator: uuid.NewGenerator(), 101 enabled: true, 102 cfg: cfg, 103 startTs: checkpoint, 104 flushIntervalInMs: cfg.MetaFlushIntervalInMs, 105 } 106 107 if m.flushIntervalInMs < redo.MinFlushIntervalInMs { 108 log.Warn("redo flush interval is too small, use default value", 109 zap.String("namespace", m.changeFeedID.Namespace), 110 zap.String("changefeed", m.changeFeedID.ID), 111 zap.Int64("interval", m.flushIntervalInMs)) 112 m.flushIntervalInMs = redo.DefaultMetaFlushIntervalInMs 113 } 114 return m 115 } 116 117 // Enabled returns whether this log manager is enabled 118 func (m *metaManager) Enabled() bool { 119 return m.enabled 120 } 121 122 // Running return whether the meta manager is initialized, 123 // which means the external storage is accessible to the meta manager. 124 func (m *metaManager) Running() bool { 125 return m.running.Load() 126 } 127 128 func (m *metaManager) preStart(ctx context.Context) error { 129 uri, err := storage.ParseRawURL(m.cfg.Storage) 130 if err != nil { 131 return err 132 } 133 // "nfs" and "local" scheme are converted to "file" scheme 134 redo.FixLocalScheme(uri) 135 // blackhole scheme is converted to "noop" scheme here, so we can use blackhole for testing 136 if redo.IsBlackholeStorage(uri.Scheme) { 137 uri, _ = storage.ParseRawURL("noop://") 138 } 139 140 extStorage, err := redo.InitExternalStorage(ctx, *uri) 141 if err != nil { 142 return err 143 } 144 m.extStorage = extStorage 145 146 m.metricFlushLogDuration = common.RedoFlushLogDurationHistogram. 147 WithLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) 148 149 err = m.preCleanupExtStorage(ctx) 150 if err != nil { 151 log.Warn("redo: pre clean redo logs fail", 152 zap.String("namespace", m.changeFeedID.Namespace), 153 zap.String("changefeed", m.changeFeedID.ID), 154 zap.Error(err)) 155 return err 156 } 157 err = m.initMeta(ctx) 158 if err != nil { 159 log.Warn("redo: init redo meta fail", 160 zap.String("namespace", m.changeFeedID.Namespace), 161 zap.String("changefeed", m.changeFeedID.ID), 162 zap.Error(err)) 163 return err 164 } 165 return nil 166 } 167 168 // Run runs bgFlushMeta and bgGC. 169 func (m *metaManager) Run(ctx context.Context, _ ...chan<- error) error { 170 if err := m.preStart(ctx); err != nil { 171 return err 172 } 173 eg, egCtx := errgroup.WithContext(ctx) 174 eg.Go(func() error { 175 return m.bgFlushMeta(egCtx) 176 }) 177 eg.Go(func() error { 178 return m.bgGC(egCtx) 179 }) 180 181 m.running.Store(true) 182 return eg.Wait() 183 } 184 185 func (m *metaManager) WaitForReady(_ context.Context) {} 186 187 func (m *metaManager) Close() {} 188 189 // UpdateMeta updates meta. 190 func (m *metaManager) UpdateMeta(checkpointTs, resolvedTs model.Ts) { 191 if ok := m.metaResolvedTs.checkAndSetUnflushed(resolvedTs); !ok { 192 log.Warn("update redo meta with a regressed resolved ts, ignore", 193 zap.Uint64("currResolvedTs", m.metaResolvedTs.getFlushed()), 194 zap.Uint64("recvResolvedTs", resolvedTs), 195 zap.String("namespace", m.changeFeedID.Namespace), 196 zap.String("changefeed", m.changeFeedID.ID)) 197 } 198 if ok := m.metaCheckpointTs.checkAndSetUnflushed(checkpointTs); !ok { 199 log.Warn("update redo meta with a regressed checkpoint ts, ignore", 200 zap.Uint64("currCheckpointTs", m.metaCheckpointTs.getFlushed()), 201 zap.Uint64("recvCheckpointTs", checkpointTs), 202 zap.String("namespace", m.changeFeedID.Namespace), 203 zap.String("changefeed", m.changeFeedID.ID)) 204 } 205 } 206 207 // GetFlushedMeta gets flushed meta. 208 func (m *metaManager) GetFlushedMeta() common.LogMeta { 209 checkpointTs := m.metaCheckpointTs.getFlushed() 210 resolvedTs := m.metaResolvedTs.getFlushed() 211 return common.LogMeta{CheckpointTs: checkpointTs, ResolvedTs: resolvedTs} 212 } 213 214 // initMeta will read the meta file from external storage and 215 // use it to initialize the meta field of the metaManager. 216 func (m *metaManager) initMeta(ctx context.Context) error { 217 select { 218 case <-ctx.Done(): 219 return errors.Trace(ctx.Err()) 220 default: 221 } 222 223 metas := []*common.LogMeta{ 224 {CheckpointTs: m.startTs, ResolvedTs: m.startTs}, 225 } 226 var toRemoveMetaFiles []string 227 err := m.extStorage.WalkDir(ctx, nil, func(path string, size int64) error { 228 log.Info("redo: meta manager walk dir", 229 zap.String("namespace", m.changeFeedID.Namespace), 230 zap.String("changefeed", m.changeFeedID.ID), 231 zap.String("path", path), zap.Int64("size", size)) 232 // TODO: use prefix to accelerate traverse operation 233 if !strings.HasSuffix(path, redo.MetaEXT) { 234 return nil 235 } 236 toRemoveMetaFiles = append(toRemoveMetaFiles, path) 237 238 data, err := m.extStorage.ReadFile(ctx, path) 239 if err != nil { 240 log.Warn("redo: read meta file failed", 241 zap.String("namespace", m.changeFeedID.Namespace), 242 zap.String("changefeed", m.changeFeedID.ID), 243 zap.String("path", path), zap.Error(err)) 244 if !util.IsNotExistInExtStorage(err) { 245 return err 246 } 247 return nil 248 } 249 var meta common.LogMeta 250 _, err = meta.UnmarshalMsg(data) 251 if err != nil { 252 log.Error("redo: unmarshal meta data failed", 253 zap.String("namespace", m.changeFeedID.Namespace), 254 zap.String("changefeed", m.changeFeedID.ID), 255 zap.Error(err), zap.ByteString("data", data)) 256 return err 257 } 258 metas = append(metas, &meta) 259 return nil 260 }) 261 if err != nil { 262 return errors.WrapError(errors.ErrRedoMetaInitialize, err) 263 } 264 265 var checkpointTs, resolvedTs uint64 266 common.ParseMeta(metas, &checkpointTs, &resolvedTs) 267 if checkpointTs == 0 || resolvedTs == 0 { 268 log.Panic("checkpointTs or resolvedTs is 0 when initializing redo meta in owner", 269 zap.String("namespace", m.changeFeedID.Namespace), 270 zap.String("changefeed", m.changeFeedID.ID), 271 zap.Uint64("checkpointTs", checkpointTs), 272 zap.Uint64("resolvedTs", resolvedTs)) 273 } 274 m.metaResolvedTs.unflushed.Store(resolvedTs) 275 m.metaCheckpointTs.unflushed.Store(checkpointTs) 276 if err := m.maybeFlushMeta(ctx); err != nil { 277 return errors.WrapError(errors.ErrRedoMetaInitialize, err) 278 } 279 280 flushedMeta := m.GetFlushedMeta() 281 log.Info("redo: meta manager flush init meta success", 282 zap.String("namespace", m.changeFeedID.Namespace), 283 zap.String("changefeed", m.changeFeedID.ID), 284 zap.Uint64("checkpointTs", flushedMeta.CheckpointTs), 285 zap.Uint64("resolvedTs", flushedMeta.ResolvedTs)) 286 287 return util.DeleteFilesInExtStorage(ctx, m.extStorage, toRemoveMetaFiles) 288 } 289 290 func (m *metaManager) preCleanupExtStorage(ctx context.Context) error { 291 deleteMarker := getDeletedChangefeedMarker(m.changeFeedID) 292 ret, err := m.extStorage.FileExists(ctx, deleteMarker) 293 if err != nil { 294 return errors.WrapError(errors.ErrExternalStorageAPI, err) 295 } 296 if !ret { 297 return nil 298 } 299 300 changefeedMatcher := getChangefeedMatcher(m.changeFeedID) 301 err = util.RemoveFilesIf(ctx, m.extStorage, func(path string) bool { 302 if path == deleteMarker || !strings.Contains(path, changefeedMatcher) { 303 return false 304 } 305 return true 306 }, nil) 307 if err != nil { 308 return err 309 } 310 311 err = m.extStorage.DeleteFile(ctx, deleteMarker) 312 if err != nil && !util.IsNotExistInExtStorage(err) { 313 return errors.WrapError(errors.ErrExternalStorageAPI, err) 314 } 315 316 return nil 317 } 318 319 // shouldRemoved remove the file which maxCommitTs in file name less than checkPointTs, since 320 // all event ts < checkPointTs already sent to sink, the log is not needed any more for recovery 321 func (m *metaManager) shouldRemoved(path string, checkPointTs uint64) bool { 322 changefeedMatcher := getChangefeedMatcher(m.changeFeedID) 323 if !strings.Contains(path, changefeedMatcher) { 324 return false 325 } 326 if filepath.Ext(path) != redo.LogEXT { 327 return false 328 } 329 330 commitTs, fileType, err := redo.ParseLogFileName(path) 331 if err != nil { 332 log.Error("parse file name failed", 333 zap.String("namespace", m.changeFeedID.Namespace), 334 zap.String("changefeed", m.changeFeedID.ID), 335 zap.String("path", path), zap.Error(err)) 336 return false 337 } 338 if fileType != redo.RedoDDLLogFileType && fileType != redo.RedoRowLogFileType { 339 log.Panic("unknown file type", 340 zap.String("namespace", m.changeFeedID.Namespace), 341 zap.String("changefeed", m.changeFeedID.ID), 342 zap.String("path", path), zap.Any("fileType", fileType)) 343 } 344 345 // if commitTs == checkPointTs, the DDL may be executed in the owner, 346 // so we should not delete it. 347 return commitTs < checkPointTs 348 } 349 350 // deleteAllLogs delete all redo logs and leave a deleted mark. 351 func (m *metaManager) deleteAllLogs(ctx context.Context) error { 352 // when one changefeed with redo enabled gets deleted, it's extStorage should always be set to not nil 353 // otherwise it should have already meet panic during changefeed running time. 354 // the extStorage may be nil in the unit test, so just set the external storage to make unit test happy. 355 if m.extStorage == nil { 356 uri, err := storage.ParseRawURL(m.cfg.Storage) 357 redo.FixLocalScheme(uri) 358 if err != nil { 359 return err 360 } 361 m.extStorage, err = redo.InitExternalStorage(ctx, *uri) 362 if err != nil { 363 return err 364 } 365 } 366 // Write deleted mark before clean any files. 367 deleteMarker := getDeletedChangefeedMarker(m.changeFeedID) 368 if err := m.extStorage.WriteFile(ctx, deleteMarker, []byte("D")); err != nil { 369 return errors.WrapError(errors.ErrExternalStorageAPI, err) 370 } 371 log.Info("redo manager write deleted mark", 372 zap.String("namespace", m.changeFeedID.Namespace), 373 zap.String("changefeed", m.changeFeedID.ID)) 374 375 changefeedMatcher := getChangefeedMatcher(m.changeFeedID) 376 return util.RemoveFilesIf(ctx, m.extStorage, func(path string) bool { 377 if path == deleteMarker || !strings.Contains(path, changefeedMatcher) { 378 return false 379 } 380 return true 381 }, nil) 382 } 383 384 func (m *metaManager) maybeFlushMeta(ctx context.Context) error { 385 hasChange, unflushed := m.prepareForFlushMeta() 386 if !hasChange { 387 // check stuck 388 if time.Since(m.lastFlushTime) > redo.FlushWarnDuration { 389 log.Warn("Redo meta has not changed for a long time, owner may be stuck", 390 zap.String("namespace", m.changeFeedID.Namespace), 391 zap.String("changefeed", m.changeFeedID.ID), 392 zap.Duration("lastFlushTime", time.Since(m.lastFlushTime)), 393 zap.Any("meta", unflushed)) 394 } 395 return nil 396 } 397 398 log.Debug("Flush redo meta", 399 zap.String("namespace", m.changeFeedID.Namespace), 400 zap.String("changefeed", m.changeFeedID.ID), 401 zap.Any("meta", unflushed)) 402 if err := m.flush(ctx, unflushed); err != nil { 403 return err 404 } 405 m.postFlushMeta(unflushed) 406 m.lastFlushTime = time.Now() 407 return nil 408 } 409 410 func (m *metaManager) prepareForFlushMeta() (bool, common.LogMeta) { 411 flushed := common.LogMeta{} 412 flushed.CheckpointTs = m.metaCheckpointTs.getFlushed() 413 flushed.ResolvedTs = m.metaResolvedTs.getFlushed() 414 415 unflushed := common.LogMeta{} 416 unflushed.CheckpointTs = m.metaCheckpointTs.getUnflushed() 417 unflushed.ResolvedTs = m.metaResolvedTs.getUnflushed() 418 419 hasChange := false 420 if flushed.CheckpointTs < unflushed.CheckpointTs || 421 flushed.ResolvedTs < unflushed.ResolvedTs { 422 hasChange = true 423 } 424 return hasChange, unflushed 425 } 426 427 func (m *metaManager) postFlushMeta(meta common.LogMeta) { 428 m.metaResolvedTs.checkAndSetFlushed(meta.ResolvedTs) 429 m.metaCheckpointTs.checkAndSetFlushed(meta.CheckpointTs) 430 } 431 432 func (m *metaManager) flush(ctx context.Context, meta common.LogMeta) error { 433 start := time.Now() 434 data, err := meta.MarshalMsg(nil) 435 if err != nil { 436 return errors.WrapError(errors.ErrMarshalFailed, err) 437 } 438 metaFile := getMetafileName(m.captureID, m.changeFeedID, m.uuidGenerator) 439 if err := m.extStorage.WriteFile(ctx, metaFile, data); err != nil { 440 log.Error("redo: meta manager flush meta write file failed", 441 zap.String("namespace", m.changeFeedID.Namespace), 442 zap.String("changefeed", m.changeFeedID.ID), 443 zap.Error(err)) 444 return errors.WrapError(errors.ErrExternalStorageAPI, err) 445 } 446 447 if m.preMetaFile != "" { 448 if m.preMetaFile == metaFile { 449 // This should only happen when use a constant uuid generator in test. 450 return nil 451 } 452 err := m.extStorage.DeleteFile(ctx, m.preMetaFile) 453 if err != nil && !util.IsNotExistInExtStorage(err) { 454 log.Error("redo: meta manager flush meta delete file failed", 455 zap.String("namespace", m.changeFeedID.Namespace), 456 zap.String("changefeed", m.changeFeedID.ID), 457 zap.Error(err)) 458 return errors.WrapError(errors.ErrExternalStorageAPI, err) 459 } 460 } 461 m.preMetaFile = metaFile 462 463 log.Debug("flush meta to s3", 464 zap.String("namespace", m.changeFeedID.Namespace), 465 zap.String("changefeed", m.changeFeedID.ID), 466 zap.String("metaFile", metaFile), 467 zap.Any("cost", time.Since(start).Milliseconds())) 468 m.metricFlushLogDuration.Observe(time.Since(start).Seconds()) 469 return nil 470 } 471 472 // Cleanup removes all redo logs of this manager, it is called when changefeed is removed 473 // only owner should call this method. 474 func (m *metaManager) Cleanup(ctx context.Context) error { 475 common.RedoWriteLogDurationHistogram. 476 DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) 477 common.RedoFlushLogDurationHistogram. 478 DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) 479 common.RedoTotalRowsCountGauge. 480 DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) 481 common.RedoWorkerBusyRatio. 482 DeleteLabelValues(m.changeFeedID.Namespace, m.changeFeedID.ID) 483 return m.deleteAllLogs(ctx) 484 } 485 486 func (m *metaManager) bgFlushMeta(egCtx context.Context) (err error) { 487 ticker := time.NewTicker(time.Duration(m.flushIntervalInMs) * time.Millisecond) 488 defer func() { 489 ticker.Stop() 490 log.Info("redo metaManager bgFlushMeta exits", 491 zap.String("namespace", m.changeFeedID.Namespace), 492 zap.String("changefeed", m.changeFeedID.ID), 493 zap.Error(err)) 494 }() 495 496 for { 497 select { 498 case <-egCtx.Done(): 499 return errors.Trace(egCtx.Err()) 500 case <-ticker.C: 501 if err := m.maybeFlushMeta(egCtx); err != nil { 502 return errors.Trace(err) 503 } 504 } 505 } 506 } 507 508 // bgGC cleans stale files before the flushed checkpoint in background. 509 func (m *metaManager) bgGC(egCtx context.Context) error { 510 ticker := time.NewTicker(time.Duration(redo.DefaultGCIntervalInMs) * time.Millisecond) 511 defer ticker.Stop() 512 513 preCkpt := uint64(0) 514 for { 515 select { 516 case <-egCtx.Done(): 517 log.Info("redo manager GC exits as context cancelled", 518 zap.String("namespace", m.changeFeedID.Namespace), 519 zap.String("changefeed", m.changeFeedID.ID)) 520 return errors.Trace(egCtx.Err()) 521 case <-ticker.C: 522 ckpt := m.metaCheckpointTs.getFlushed() 523 if ckpt == preCkpt { 524 continue 525 } 526 preCkpt = ckpt 527 log.Debug("redo manager GC is triggered", 528 zap.Uint64("checkpointTs", ckpt), 529 zap.String("namespace", m.changeFeedID.Namespace), 530 zap.String("changefeed", m.changeFeedID.ID)) 531 err := util.RemoveFilesIf(egCtx, m.extStorage, func(path string) bool { 532 return m.shouldRemoved(path, ckpt) 533 }, nil) 534 if err != nil { 535 log.Warn("redo manager log GC fail", 536 zap.String("namespace", m.changeFeedID.Namespace), 537 zap.String("changefeed", m.changeFeedID.ID), zap.Error(err)) 538 return errors.Trace(err) 539 } 540 } 541 } 542 } 543 544 func getMetafileName( 545 captureID model.CaptureID, 546 changeFeedID model.ChangeFeedID, 547 uuidGenerator uuid.Generator, 548 ) string { 549 return fmt.Sprintf(redo.RedoMetaFileFormat, captureID, 550 changeFeedID.Namespace, changeFeedID.ID, 551 redo.RedoMetaFileType, uuidGenerator.NewString(), redo.MetaEXT) 552 } 553 554 func getChangefeedMatcher(changeFeedID model.ChangeFeedID) string { 555 if changeFeedID.Namespace == "default" { 556 return fmt.Sprintf("_%s_", changeFeedID.ID) 557 } 558 return fmt.Sprintf("_%s_%s_", changeFeedID.Namespace, changeFeedID.ID) 559 } 560 561 func getDeletedChangefeedMarker(changeFeedID model.ChangeFeedID) string { 562 if changeFeedID.Namespace == model.DefaultNamespace { 563 return fmt.Sprintf("delete_%s", changeFeedID.ID) 564 } 565 return fmt.Sprintf("delete_%s_%s", changeFeedID.Namespace, changeFeedID.ID) 566 }