github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/owner/changefeed.go (about) 1 // Copyright 2021 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 owner 15 16 import ( 17 "context" 18 "fmt" 19 "math" 20 "strings" 21 "sync" 22 "time" 23 24 "github.com/pingcap/errors" 25 "github.com/pingcap/failpoint" 26 "github.com/pingcap/log" 27 "github.com/pingcap/tidb/pkg/errno" 28 "github.com/pingcap/tiflow/cdc/async" 29 "github.com/pingcap/tiflow/cdc/entry" 30 "github.com/pingcap/tiflow/cdc/model" 31 "github.com/pingcap/tiflow/cdc/puller" 32 "github.com/pingcap/tiflow/cdc/redo" 33 "github.com/pingcap/tiflow/cdc/scheduler" 34 "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" 35 "github.com/pingcap/tiflow/cdc/vars" 36 "github.com/pingcap/tiflow/pkg/config" 37 cerror "github.com/pingcap/tiflow/pkg/errors" 38 pfilter "github.com/pingcap/tiflow/pkg/filter" 39 "github.com/pingcap/tiflow/pkg/pdutil" 40 redoCfg "github.com/pingcap/tiflow/pkg/redo" 41 "github.com/pingcap/tiflow/pkg/sink/observer" 42 "github.com/pingcap/tiflow/pkg/txnutil/gc" 43 "github.com/pingcap/tiflow/pkg/upstream" 44 "github.com/pingcap/tiflow/pkg/util" 45 "github.com/prometheus/client_golang/prometheus" 46 "github.com/tikv/client-go/v2/oracle" 47 "go.uber.org/atomic" 48 "go.uber.org/zap" 49 ) 50 51 // Changefeed is the tick logic of changefeed. 52 type Changefeed interface { 53 // Tick is called periodically to drive the changefeed's internal logic. 54 // The main logic of changefeed is in this function, including the calculation of many kinds of ts, 55 // maintain table components, error handling, etc. 56 // 57 // It can be called in etcd ticks, so it should never be blocked. 58 // Tick Returns: checkpointTs, minTableBarrierTs 59 Tick(context.Context, *model.ChangeFeedInfo, 60 *model.ChangeFeedStatus, 61 map[model.CaptureID]*model.CaptureInfo) (model.Ts, model.Ts) 62 63 // Close closes the changefeed. 64 Close(ctx context.Context) 65 66 // GetScheduler returns the scheduler of this changefeed. 67 GetScheduler() scheduler.Scheduler 68 } 69 70 var _ Changefeed = (*changefeed)(nil) 71 72 // newScheduler creates a new scheduler from context. 73 // This function is factored out to facilitate unit testing. 74 func newScheduler( 75 ctx context.Context, 76 changeFeedID model.ChangeFeedID, 77 up *upstream.Upstream, epoch uint64, 78 cfg *config.SchedulerConfig, 79 redoMetaManager redo.MetaManager, 80 globalVars *vars.GlobalVars, 81 ) (scheduler.Scheduler, error) { 82 messageServer := globalVars.MessageServer 83 messageRouter := globalVars.MessageRouter 84 ownerRev := globalVars.OwnerRevision 85 captureID := globalVars.CaptureInfo.ID 86 ret, err := scheduler.NewScheduler( 87 ctx, captureID, changeFeedID, messageServer, messageRouter, ownerRev, epoch, up, cfg, redoMetaManager) 88 return ret, errors.Trace(err) 89 } 90 91 type changefeed struct { 92 id model.ChangeFeedID 93 94 upstream *upstream.Upstream 95 cfg *config.SchedulerConfig 96 scheduler scheduler.Scheduler 97 globalVars *vars.GlobalVars 98 // barriers will be created when a changefeed is initialized 99 // and will be destroyed when a changefeed is closed. 100 barriers *barriers 101 feedStateManager FeedStateManager 102 resolvedTs model.Ts 103 104 // lastSyncedTs is the lastest resolvedTs that has been synced to downstream. 105 // pullerResolvedTs is the minimum resolvedTs of all pullers. 106 // we don't need to initialize lastSyncedTs and pullerResolvedTs specially 107 // because it will be updated in tick. 108 lastSyncedTs model.Ts 109 pullerResolvedTs model.Ts 110 111 // ddl related fields 112 ddlManager *ddlManager 113 redoDDLMgr redo.DDLManager 114 redoMetaMgr redo.MetaManager 115 116 schema entry.SchemaStorage 117 ddlSink DDLSink 118 ddlPuller puller.DDLPuller 119 // The changefeed will start a backend goroutine in the function `initialize` 120 // for DDLPuller and redo manager. `wg` is used to manage this backend goroutine. 121 wg sync.WaitGroup 122 123 // state related fields 124 initialized *atomic.Bool 125 initializer *async.Initializer 126 127 // isRemoved is true if the changefeed is removed, 128 // which means it will be removed from memory forever 129 isRemoved bool 130 // isReleased is true if the changefeed's resources were released, 131 // but it will still be kept in the memory, and it will be check 132 // in every tick. Such as the changefeed that is stopped or encountered an error. 133 isReleased bool 134 errCh chan error 135 warningCh chan error 136 // cancel the running goroutine start by `DDLPuller` 137 cancel context.CancelFunc 138 139 metricsChangefeedCheckpointTsGauge prometheus.Gauge 140 metricsChangefeedCheckpointTsLagGauge prometheus.Gauge 141 metricsChangefeedCheckpointLagDuration prometheus.Observer 142 143 metricsChangefeedResolvedTsGauge prometheus.Gauge 144 metricsChangefeedResolvedTsLagGauge prometheus.Gauge 145 metricsChangefeedResolvedTsLagDuration prometheus.Observer 146 metricsCurrentPDTsGauge prometheus.Gauge 147 148 metricsChangefeedBarrierTsGauge prometheus.Gauge 149 metricsChangefeedTickDuration prometheus.Observer 150 151 metricsChangefeedCreateTimeGuage prometheus.Gauge 152 metricsChangefeedRestartTimeGauge prometheus.Gauge 153 154 downstreamObserver observer.Observer 155 observerLastTick *atomic.Time 156 157 newDDLPuller func(ctx context.Context, 158 up *upstream.Upstream, 159 startTs uint64, 160 changefeed model.ChangeFeedID, 161 schemaStorage entry.SchemaStorage, 162 filter pfilter.Filter, 163 ) puller.DDLPuller 164 165 newSink func( 166 changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, 167 reportError func(err error), reportWarning func(err error), 168 ) DDLSink 169 170 newScheduler func( 171 ctx context.Context, changefeedID model.ChangeFeedID, 172 up *upstream.Upstream, epoch uint64, cfg *config.SchedulerConfig, 173 redoMetaManager redo.MetaManager, globalVars *vars.GlobalVars, 174 ) (scheduler.Scheduler, error) 175 176 newDownstreamObserver func( 177 ctx context.Context, 178 changefeedID model.ChangeFeedID, 179 sinkURIStr string, replCfg *config.ReplicaConfig, 180 opts ...observer.NewObserverOption, 181 ) (observer.Observer, error) 182 183 lastDDLTs uint64 // Timestamp of the last executed DDL. Only used for tests. 184 185 // The latest changefeed info and status from meta storage. they are updated in every Tick. 186 latestInfo *model.ChangeFeedInfo 187 latestStatus *model.ChangeFeedStatus 188 } 189 190 func (c *changefeed) GetScheduler() scheduler.Scheduler { 191 return c.scheduler 192 } 193 194 // NewChangefeed creates a new changefeed. 195 func NewChangefeed( 196 id model.ChangeFeedID, 197 cfInfo *model.ChangeFeedInfo, 198 cfStatus *model.ChangeFeedStatus, 199 feedStateManager FeedStateManager, 200 up *upstream.Upstream, 201 cfg *config.SchedulerConfig, 202 globalVars *vars.GlobalVars, 203 ) *changefeed { 204 c := &changefeed{ 205 id: id, 206 latestInfo: cfInfo, 207 latestStatus: cfStatus, 208 // The scheduler will be created lazily. 209 scheduler: nil, 210 barriers: newBarriers(), 211 feedStateManager: feedStateManager, 212 upstream: up, 213 214 errCh: make(chan error, defaultErrChSize), 215 warningCh: make(chan error, defaultErrChSize), 216 cancel: func() {}, 217 218 newDDLPuller: puller.NewDDLPuller, 219 newSink: newDDLSink, 220 newDownstreamObserver: observer.NewObserver, 221 initialized: atomic.NewBool(false), 222 223 globalVars: globalVars, 224 } 225 c.newScheduler = newScheduler 226 c.cfg = cfg 227 c.initializer = async.NewInitializer() 228 return c 229 } 230 231 func newChangefeed4Test( 232 id model.ChangeFeedID, 233 cfInfo *model.ChangeFeedInfo, 234 cfStatus *model.ChangeFeedStatus, 235 cfstateManager FeedStateManager, up *upstream.Upstream, 236 newDDLPuller func(ctx context.Context, 237 up *upstream.Upstream, 238 startTs uint64, 239 changefeed model.ChangeFeedID, 240 schemaStorage entry.SchemaStorage, 241 filter pfilter.Filter, 242 ) puller.DDLPuller, 243 newSink func( 244 changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, 245 reportError func(err error), reportWarning func(err error), 246 ) DDLSink, 247 newScheduler func(ctx context.Context, id model.ChangeFeedID, 248 up *upstream.Upstream, epoch uint64, cfg *config.SchedulerConfig, redoMetaManager redo.MetaManager, 249 globalVars *vars.GlobalVars, 250 ) (scheduler.Scheduler, error), 251 newDownstreamObserver func( 252 ctx context.Context, 253 changefeedID model.ChangeFeedID, 254 sinkURIStr string, replCfg *config.ReplicaConfig, 255 opts ...observer.NewObserverOption, 256 ) (observer.Observer, error), 257 globalVars *vars.GlobalVars, 258 ) *changefeed { 259 cfg := config.NewDefaultSchedulerConfig() 260 c := NewChangefeed(id, cfInfo, cfStatus, cfstateManager, up, cfg, globalVars) 261 c.newDDLPuller = newDDLPuller 262 c.newSink = newSink 263 c.newScheduler = newScheduler 264 c.newDownstreamObserver = newDownstreamObserver 265 return c 266 } 267 268 func (c *changefeed) Tick(ctx context.Context, 269 cfInfo *model.ChangeFeedInfo, 270 cfStatus *model.ChangeFeedStatus, 271 captures map[model.CaptureID]*model.CaptureInfo, 272 ) (model.Ts, model.Ts) { 273 startTime := time.Now() 274 c.latestInfo = cfInfo 275 c.latestStatus = cfStatus 276 // Handle all internal warnings. 277 noMoreWarnings := false 278 for !noMoreWarnings { 279 select { 280 case err := <-c.warningCh: 281 c.handleWarning(err) 282 default: 283 noMoreWarnings = true 284 } 285 } 286 287 if skip, err := c.checkUpstream(); skip { 288 if err != nil { 289 c.handleErr(ctx, err) 290 } 291 return 0, 0 292 } 293 294 checkpointTs, minTableBarrierTs, err := c.tick(ctx, captures, cfInfo, cfStatus) 295 296 // The tick duration is recorded only if changefeed has completed initialization 297 if c.initialized.Load() { 298 costTime := time.Since(startTime) 299 if costTime > changefeedLogsWarnDuration { 300 log.Warn("changefeed tick took too long", 301 zap.String("namespace", c.id.Namespace), 302 zap.String("changefeed", c.id.ID), 303 zap.Duration("duration", costTime)) 304 } 305 c.metricsChangefeedTickDuration.Observe(costTime.Seconds()) 306 } 307 308 if err != nil { 309 log.Error("changefeed tick failed", zap.Error(err)) 310 c.handleErr(ctx, err) 311 } 312 return checkpointTs, minTableBarrierTs 313 } 314 315 func (c *changefeed) Throw(ctx context.Context) func(error) { 316 return func(err error) { 317 select { 318 case <-ctx.Done(): 319 case c.errCh <- errors.Trace(err): 320 } 321 } 322 } 323 324 func (c *changefeed) handleErr(ctx context.Context, err error) { 325 log.Error("an error occurred in Owner", 326 zap.String("namespace", c.id.Namespace), 327 zap.String("changefeed", c.id.ID), zap.Error(err)) 328 var code string 329 if rfcCode, ok := cerror.RFCCode(err); ok { 330 code = string(rfcCode) 331 } else { 332 code = string(cerror.ErrOwnerUnknown.RFCCode()) 333 } 334 c.feedStateManager.HandleError(&model.RunningError{ 335 Time: time.Now(), 336 Addr: config.GetGlobalServerConfig().AdvertiseAddr, 337 Code: code, 338 Message: err.Error(), 339 }) 340 c.releaseResources(ctx) 341 } 342 343 func (c *changefeed) handleWarning(err error) { 344 log.Warn("an warning occurred in Owner", 345 zap.String("namespace", c.id.Namespace), 346 zap.String("changefeed", c.id.ID), zap.Error(err)) 347 var code string 348 if rfcCode, ok := cerror.RFCCode(err); ok { 349 code = string(rfcCode) 350 } else { 351 code = string(cerror.ErrOwnerUnknown.RFCCode()) 352 } 353 354 c.feedStateManager.HandleWarning(&model.RunningError{ 355 Time: time.Now(), 356 Addr: config.GetGlobalServerConfig().AdvertiseAddr, 357 Code: code, 358 Message: err.Error(), 359 }) 360 } 361 362 func (c *changefeed) checkStaleCheckpointTs( 363 ctx context.Context, checkpointTs uint64, 364 cfInfo *model.ChangeFeedInfo, 365 ) error { 366 if cfInfo.NeedBlockGC() { 367 failpoint.Inject("InjectChangefeedFastFailError", func() error { 368 return cerror.ErrStartTsBeforeGC.FastGen("InjectChangefeedFastFailError") 369 }) 370 if err := c.upstream.GCManager.CheckStaleCheckpointTs(ctx, c.id, checkpointTs); err != nil { 371 return errors.Trace(err) 372 } 373 } 374 return nil 375 } 376 377 // tick is the main logic of changefeed. 378 // tick returns the checkpointTs and minTableBarrierTs. 379 func (c *changefeed) tick(ctx context.Context, 380 captures map[model.CaptureID]*model.CaptureInfo, 381 cfInfo *model.ChangeFeedInfo, 382 cfStatus *model.ChangeFeedStatus, 383 ) (model.Ts, model.Ts, error) { 384 adminJobPending := c.feedStateManager.Tick(c.resolvedTs, cfStatus, cfInfo) 385 preCheckpointTs := cfInfo.GetCheckpointTs(cfStatus) 386 // checkStaleCheckpointTs must be called before `feedStateManager.ShouldRunning()` 387 // to ensure all changefeeds, no matter whether they are running or not, will be checked. 388 if err := c.checkStaleCheckpointTs(ctx, preCheckpointTs, cfInfo); err != nil { 389 return 0, 0, errors.Trace(err) 390 } 391 392 if !c.feedStateManager.ShouldRunning() { 393 c.isRemoved = c.feedStateManager.ShouldRemoved() 394 c.releaseResources(ctx) 395 return 0, 0, nil 396 } 397 398 if adminJobPending { 399 return 0, 0, nil 400 } 401 402 if !c.initialized.Load() { 403 initialized, err := c.initializer.TryInitialize(ctx, 404 func(ctx context.Context) error { 405 return c.initialize(ctx, cfInfo, cfStatus) 406 }, 407 c.globalVars.ChangefeedThreadPool) 408 if err != nil { 409 return 0, 0, errors.Trace(err) 410 } 411 if !initialized { 412 return 0, 0, nil 413 } 414 } 415 416 select { 417 case err := <-c.errCh: 418 return 0, 0, errors.Trace(err) 419 default: 420 } 421 422 if c.redoMetaMgr.Enabled() { 423 if !c.redoMetaMgr.Running() { 424 return 0, 0, nil 425 } 426 } 427 428 allPhysicalTables, barrier, err := c.ddlManager.tick(ctx, preCheckpointTs) 429 if err != nil { 430 return 0, 0, errors.Trace(err) 431 } 432 433 err = c.handleBarrier(ctx, cfInfo, cfStatus, barrier) 434 if err != nil { 435 return 0, 0, errors.Trace(err) 436 } 437 438 log.Debug("owner handles barrier", 439 zap.String("namespace", c.id.Namespace), 440 zap.String("changefeed", c.id.ID), 441 zap.Uint64("preCheckpointTs", preCheckpointTs), 442 zap.Uint64("preResolvedTs", c.resolvedTs), 443 zap.Uint64("globalBarrierTs", barrier.GlobalBarrierTs), 444 zap.Uint64("minTableBarrierTs", barrier.MinTableBarrierTs), 445 zap.Any("tableBarrier", barrier.TableBarriers)) 446 447 if barrier.GlobalBarrierTs < preCheckpointTs { 448 // This condition implies that the DDL resolved-ts has not yet reached checkpointTs, 449 // which implies that it would be premature to schedule tables or to update status. 450 // So we return here. 451 return 0, 0, nil 452 } 453 454 watermark, err := c.scheduler.Tick( 455 ctx, preCheckpointTs, allPhysicalTables, captures, 456 barrier) 457 if err != nil { 458 return 0, 0, errors.Trace(err) 459 } 460 461 if watermark.LastSyncedTs != scheduler.CheckpointCannotProceed { 462 if c.lastSyncedTs < watermark.LastSyncedTs { 463 c.lastSyncedTs = watermark.LastSyncedTs 464 } else if c.lastSyncedTs > watermark.LastSyncedTs { 465 log.Warn("LastSyncedTs should not be greater than newLastSyncedTs", 466 zap.Uint64("c.LastSyncedTs", c.lastSyncedTs), 467 zap.Uint64("newLastSyncedTs", watermark.LastSyncedTs)) 468 } 469 } 470 471 if watermark.PullerResolvedTs != scheduler.CheckpointCannotProceed && watermark.PullerResolvedTs != math.MaxUint64 { 472 if watermark.PullerResolvedTs > c.pullerResolvedTs { 473 c.pullerResolvedTs = watermark.PullerResolvedTs 474 } else if watermark.PullerResolvedTs < c.pullerResolvedTs { 475 log.Warn("the newPullerResolvedTs should not be smaller than c.pullerResolvedTs", 476 zap.Uint64("c.pullerResolvedTs", c.pullerResolvedTs), 477 zap.Uint64("newPullerResolvedTs", watermark.PullerResolvedTs)) 478 } 479 } 480 481 pdTime := c.upstream.PDClock.CurrentTime() 482 currentTs := oracle.GetPhysical(pdTime) 483 484 // CheckpointCannotProceed implies that not all tables are being replicated normally, 485 // so in that case there is no need to advance the global watermarks. 486 if watermark.CheckpointTs == scheduler.CheckpointCannotProceed { 487 if cfStatus != nil { 488 // We should keep the metrics updated even if the scheduler cannot 489 // advance the watermarks for now. 490 c.updateMetrics(currentTs, cfStatus.CheckpointTs, c.resolvedTs) 491 } 492 return 0, 0, nil 493 } 494 495 log.Debug("owner prepares to update status", 496 zap.Uint64("prevResolvedTs", c.resolvedTs), 497 zap.Uint64("newResolvedTs", watermark.ResolvedTs), 498 zap.Uint64("newCheckpointTs", watermark.CheckpointTs), 499 zap.String("namespace", c.id.Namespace), 500 zap.String("changefeed", c.id.ID)) 501 // resolvedTs should never regress. 502 if watermark.ResolvedTs > c.resolvedTs { 503 c.resolvedTs = watermark.ResolvedTs 504 } 505 506 // MinTableBarrierTs should never regress 507 if barrier.MinTableBarrierTs < cfStatus.MinTableBarrierTs { 508 barrier.MinTableBarrierTs = cfStatus.MinTableBarrierTs 509 } 510 511 failpoint.Inject("ChangefeedOwnerDontUpdateCheckpoint", func() { 512 if c.lastDDLTs != 0 && cfStatus.CheckpointTs >= c.lastDDLTs { 513 log.Info("owner won't update checkpoint because of failpoint", 514 zap.String("namespace", c.id.Namespace), 515 zap.String("changefeed", c.id.ID), 516 zap.Uint64("keepCheckpoint", cfStatus.CheckpointTs), 517 zap.Uint64("skipCheckpoint", watermark.CheckpointTs)) 518 watermark.CheckpointTs = cfStatus.CheckpointTs 519 } 520 }) 521 522 failpoint.Inject("ChangefeedOwnerNotUpdateCheckpoint", func() { 523 watermark.CheckpointTs = cfStatus.CheckpointTs 524 }) 525 526 c.updateMetrics(currentTs, watermark.CheckpointTs, c.resolvedTs) 527 c.tickDownstreamObserver(ctx) 528 529 return watermark.CheckpointTs, barrier.MinTableBarrierTs, nil 530 } 531 532 func (c *changefeed) initialize(ctx context.Context, 533 cfInfo *model.ChangeFeedInfo, 534 cfStatus *model.ChangeFeedStatus, 535 ) (err error) { 536 if c.initialized.Load() || cfStatus == nil { 537 // If `c.latestStatus` is nil it means the changefeed struct is just created, it needs to 538 // 1. use startTs as checkpointTs and resolvedTs, if it's a new created changefeed; or 539 // 2. load checkpointTs and resolvedTs from etcd, if it's an existing changefeed. 540 // And then it can continue to initialize. 541 return nil 542 } 543 c.isReleased = false 544 545 // clean the errCh 546 // When the changefeed is resumed after being stopped, the changefeed instance will be reused, 547 // So we should make sure that the errCh is empty when the changefeed is restarting 548 LOOP1: 549 for { 550 select { 551 case <-c.errCh: 552 default: 553 break LOOP1 554 } 555 } 556 LOOP2: 557 for { 558 select { 559 case <-c.warningCh: 560 default: 561 break LOOP2 562 } 563 } 564 565 checkpointTs := cfStatus.CheckpointTs 566 if c.resolvedTs == 0 { 567 c.resolvedTs = checkpointTs 568 } 569 570 minTableBarrierTs := cfStatus.MinTableBarrierTs 571 572 failpoint.Inject("NewChangefeedNoRetryError", func() { 573 failpoint.Return(cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs)) 574 }) 575 failpoint.Inject("NewChangefeedRetryError", func() { 576 failpoint.Return(errors.New("failpoint injected retriable error")) 577 }) 578 579 if cfInfo.Config.CheckGCSafePoint { 580 // Check TiDB GC safepoint does not exceed the checkpoint. 581 // 582 // We update TTL to 10 minutes, 583 // 1. to delete the service GC safepoint effectively, 584 // 2. in case owner update TiCDC service GC safepoint fails. 585 // 586 // Also, it unblocks TiDB GC, because the service GC safepoint is set to 587 // 1 hour TTL during creating changefeed. 588 // 589 // See more gc doc. 590 ensureTTL := int64(10 * 60) 591 err = gc.EnsureChangefeedStartTsSafety( 592 ctx, c.upstream.PDClient, 593 c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceInitializing), 594 c.id, ensureTTL, checkpointTs) 595 if err != nil { 596 return errors.Trace(err) 597 } 598 // clean service GC safepoint '-creating-' and '-resuming-' if there are any. 599 err = gc.UndoEnsureChangefeedStartTsSafety( 600 ctx, c.upstream.PDClient, 601 c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceCreating), 602 c.id, 603 ) 604 if err != nil { 605 return errors.Trace(err) 606 } 607 err = gc.UndoEnsureChangefeedStartTsSafety( 608 ctx, c.upstream.PDClient, 609 c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceResuming), 610 c.id, 611 ) 612 if err != nil { 613 return errors.Trace(err) 614 } 615 } 616 617 var ddlStartTs model.Ts 618 // This means there was a ddl job when the changefeed was paused. 619 // We don't know whether the ddl job is finished or not, so we need to 620 // start the ddl puller from the `checkpointTs-1` to execute the ddl job 621 // again. 622 // FIXME: TiCDC can't handle some ddl jobs correctly in this situation. 623 // For example, if the ddl job is `add index`, TiCDC will execute the ddl 624 // job again and cause the index to be added twice. We need to fix this 625 // problem in the future. See:https://github.com/pingcap/tiflow/issues/2543 626 if checkpointTs == minTableBarrierTs { 627 ddlStartTs = checkpointTs - 1 628 } else { 629 ddlStartTs = checkpointTs 630 } 631 632 c.barriers = newBarriers() 633 if util.GetOrZero(cfInfo.Config.EnableSyncPoint) { 634 c.barriers.Update(syncPointBarrier, c.resolvedTs) 635 } 636 c.barriers.Update(finishBarrier, cfInfo.GetTargetTs()) 637 638 filter, err := pfilter.NewFilter(cfInfo.Config, "") 639 if err != nil { 640 return errors.Trace(err) 641 } 642 c.schema, err = entry.NewSchemaStorage( 643 c.upstream.KVStorage, ddlStartTs, 644 cfInfo.Config.ForceReplicate, c.id, util.RoleOwner, filter) 645 if err != nil { 646 return errors.Trace(err) 647 } 648 649 cancelCtx, cancel := context.WithCancel(ctx) 650 c.cancel = cancel 651 652 sourceID, err := pdutil.GetSourceID(ctx, c.upstream.PDClient) 653 if err != nil { 654 return errors.Trace(err) 655 } 656 cfInfo.Config.Sink.TiDBSourceID = sourceID 657 log.Info("get sourceID from PD", zap.Uint64("sourceID", sourceID), zap.Stringer("changefeedID", c.id)) 658 659 c.ddlSink = c.newSink(c.id, cfInfo, c.Throw(ctx), func(err error) { 660 select { 661 case <-ctx.Done(): 662 case c.warningCh <- err: 663 } 664 }) 665 c.ddlSink.run(cancelCtx) 666 667 c.ddlPuller = c.newDDLPuller(cancelCtx, c.upstream, ddlStartTs, c.id, c.schema, filter) 668 c.wg.Add(1) 669 go func() { 670 defer c.wg.Done() 671 c.Throw(ctx)(c.ddlPuller.Run(cancelCtx)) 672 }() 673 674 c.downstreamObserver, err = c.newDownstreamObserver(ctx, c.id, cfInfo.SinkURI, cfInfo.Config) 675 if err != nil { 676 return err 677 } 678 c.observerLastTick = atomic.NewTime(time.Time{}) 679 680 c.redoDDLMgr = redo.NewDDLManager(c.id, cfInfo.Config.Consistent, ddlStartTs) 681 if c.redoDDLMgr.Enabled() { 682 c.wg.Add(1) 683 go func() { 684 defer c.wg.Done() 685 c.Throw(ctx)(c.redoDDLMgr.Run(cancelCtx)) 686 }() 687 } 688 689 c.redoMetaMgr = redo.NewMetaManager(c.id, cfInfo.Config.Consistent, checkpointTs) 690 if c.redoMetaMgr.Enabled() { 691 c.wg.Add(1) 692 go func() { 693 defer c.wg.Done() 694 c.Throw(ctx)(c.redoMetaMgr.Run(cancelCtx)) 695 }() 696 log.Info("owner creates redo manager", 697 zap.String("namespace", c.id.Namespace), 698 zap.String("changefeed", c.id.ID)) 699 } 700 701 c.ddlManager = newDDLManager( 702 c.id, 703 ddlStartTs, 704 cfStatus.CheckpointTs, 705 c.ddlSink, 706 filter, 707 c.ddlPuller, 708 c.schema, 709 c.redoDDLMgr, 710 c.redoMetaMgr, 711 util.GetOrZero(cfInfo.Config.BDRMode)) 712 713 // create scheduler 714 cfg := *c.cfg 715 cfg.ChangefeedSettings = cfInfo.Config.Scheduler 716 epoch := cfInfo.Epoch 717 c.scheduler, err = c.newScheduler(ctx, c.id, c.upstream, epoch, &cfg, c.redoMetaMgr, c.globalVars) 718 if err != nil { 719 return errors.Trace(err) 720 } 721 722 c.initMetrics() 723 724 c.initialized.Store(true) 725 c.metricsChangefeedCreateTimeGuage.Set(float64(oracle.GetPhysical(cfInfo.CreateTime))) 726 c.metricsChangefeedRestartTimeGauge.Set(float64(oracle.GetPhysical(time.Now()))) 727 log.Info("changefeed initialized", 728 zap.String("namespace", c.id.Namespace), 729 zap.String("changefeed", c.id.ID), 730 zap.Uint64("changefeedEpoch", epoch), 731 zap.Uint64("checkpointTs", checkpointTs), 732 zap.Uint64("resolvedTs", c.resolvedTs), 733 zap.String("info", cfInfo.String())) 734 735 return nil 736 } 737 738 func (c *changefeed) initMetrics() { 739 c.metricsChangefeedCheckpointTsGauge = changefeedCheckpointTsGauge. 740 WithLabelValues(c.id.Namespace, c.id.ID) 741 c.metricsChangefeedCheckpointTsLagGauge = changefeedCheckpointTsLagGauge. 742 WithLabelValues(c.id.Namespace, c.id.ID) 743 c.metricsChangefeedCheckpointLagDuration = changefeedCheckpointLagDuration. 744 WithLabelValues(c.id.Namespace, c.id.ID) 745 746 c.metricsChangefeedResolvedTsGauge = changefeedResolvedTsGauge. 747 WithLabelValues(c.id.Namespace, c.id.ID) 748 c.metricsChangefeedResolvedTsLagGauge = changefeedResolvedTsLagGauge. 749 WithLabelValues(c.id.Namespace, c.id.ID) 750 c.metricsChangefeedResolvedTsLagDuration = changefeedResolvedTsLagDuration. 751 WithLabelValues(c.id.Namespace, c.id.ID) 752 c.metricsCurrentPDTsGauge = currentPDTsGauge.WithLabelValues(c.id.Namespace, c.id.ID) 753 754 c.metricsChangefeedBarrierTsGauge = changefeedBarrierTsGauge. 755 WithLabelValues(c.id.Namespace, c.id.ID) 756 c.metricsChangefeedTickDuration = changefeedTickDuration. 757 WithLabelValues(c.id.Namespace, c.id.ID) 758 759 c.metricsChangefeedCreateTimeGuage = changefeedStartTimeGauge. 760 WithLabelValues(c.id.Namespace, c.id.ID, "create") 761 c.metricsChangefeedRestartTimeGauge = changefeedStartTimeGauge. 762 WithLabelValues(c.id.Namespace, c.id.ID, "restart") 763 } 764 765 // releaseResources is idempotent. 766 func (c *changefeed) releaseResources(ctx context.Context) { 767 c.initializer.Terminate() 768 c.cleanupMetrics() 769 if c.isReleased { 770 return 771 } 772 // Must clean redo manager before calling cancel, otherwise 773 // the manager can be closed internally. 774 c.cleanupRedoManager(ctx, c.latestInfo) 775 c.cleanupChangefeedServiceGCSafePoints(ctx) 776 777 if c.cancel != nil { 778 c.cancel() 779 } 780 c.cancel = func() {} 781 782 if c.ddlPuller != nil { 783 c.ddlPuller.Close() 784 } 785 c.wg.Wait() 786 787 if c.ddlSink != nil { 788 canceledCtx, cancel := context.WithCancel(context.Background()) 789 cancel() 790 // TODO(dongmen): remove ctx from func ddlSink.close(), it is useless. 791 // We don't need to wait ddlSink Close, pass a canceled context is ok 792 if err := c.ddlSink.close(canceledCtx); err != nil { 793 log.Warn("owner close ddlSink failed", 794 zap.String("namespace", c.id.Namespace), 795 zap.String("changefeed", c.id.ID), 796 zap.Error(err)) 797 } 798 } 799 800 if c.scheduler != nil { 801 c.scheduler.Close(ctx) 802 c.scheduler = nil 803 } 804 if c.downstreamObserver != nil { 805 _ = c.downstreamObserver.Close() 806 } 807 808 c.schema = nil 809 c.barriers = nil 810 c.resolvedTs = 0 811 c.initialized.Store(false) 812 c.isReleased = true 813 814 log.Info("changefeed closed", 815 zap.String("namespace", c.id.Namespace), 816 zap.String("changefeed", c.id.ID), 817 zap.Bool("isRemoved", c.isRemoved)) 818 } 819 820 func (c *changefeed) cleanupMetrics() { 821 changefeedCheckpointTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 822 changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 823 changefeedCheckpointLagDuration.DeleteLabelValues(c.id.Namespace, c.id.ID) 824 c.metricsChangefeedCheckpointTsGauge = nil 825 c.metricsChangefeedCheckpointTsLagGauge = nil 826 c.metricsChangefeedCheckpointLagDuration = nil 827 828 changefeedResolvedTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 829 changefeedResolvedTsLagGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 830 changefeedResolvedTsLagDuration.DeleteLabelValues(c.id.Namespace, c.id.ID) 831 currentPDTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 832 c.metricsChangefeedResolvedTsGauge = nil 833 c.metricsChangefeedResolvedTsLagGauge = nil 834 c.metricsChangefeedResolvedTsLagDuration = nil 835 c.metricsCurrentPDTsGauge = nil 836 837 changefeedTickDuration.DeleteLabelValues(c.id.Namespace, c.id.ID) 838 c.metricsChangefeedTickDuration = nil 839 840 changefeedBarrierTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 841 c.metricsChangefeedBarrierTsGauge = nil 842 843 if c.isRemoved { 844 changefeedStatusGauge.DeleteLabelValues(c.id.Namespace, c.id.ID) 845 changefeedCheckpointTsGauge.DeleteLabelValues(c.id.Namespace, c.id.ID, "create") 846 changefeedCheckpointTsLagGauge.DeleteLabelValues(c.id.Namespace, c.id.ID, "restart") 847 } 848 } 849 850 // cleanup redo logs if changefeed is removed and redo log is enabled 851 func (c *changefeed) cleanupRedoManager(ctx context.Context, cfInfo *model.ChangeFeedInfo) { 852 if c.isRemoved { 853 if cfInfo == nil || cfInfo.Config == nil || 854 cfInfo.Config.Consistent == nil { 855 log.Warn("changefeed is removed, but state is not complete", zap.Any("info", cfInfo)) 856 return 857 } 858 if !redoCfg.IsConsistentEnabled(cfInfo.Config.Consistent.Level) { 859 return 860 } 861 // when removing a paused changefeed, the redo manager is nil, create a new one 862 if c.redoMetaMgr == nil { 863 c.redoMetaMgr = redo.NewMetaManager(c.id, cfInfo.Config.Consistent, 0) 864 } 865 err := c.redoMetaMgr.Cleanup(ctx) 866 if err != nil { 867 log.Error("cleanup redo logs failed", zap.String("changefeed", c.id.ID), zap.Error(err)) 868 } 869 } 870 } 871 872 func (c *changefeed) cleanupChangefeedServiceGCSafePoints(ctx context.Context) { 873 if !c.isRemoved { 874 return 875 } 876 877 serviceIDs := []string{ 878 c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceCreating), 879 c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceResuming), 880 c.globalVars.EtcdClient.GetEnsureGCServiceID(gc.EnsureGCServiceInitializing), 881 } 882 883 for _, serviceID := range serviceIDs { 884 err := gc.UndoEnsureChangefeedStartTsSafety( 885 ctx, 886 c.upstream.PDClient, 887 serviceID, 888 c.id) 889 if err != nil { 890 log.Error("failed to remove gc safepoint", 891 zap.String("namespace", c.id.Namespace), 892 zap.String("changefeed", c.id.ID), 893 zap.String("serviceID", serviceID)) 894 } 895 } 896 } 897 898 // handleBarrier calculates the barrierTs of the changefeed. 899 // barrierTs is used to control the data that can be flush to downstream. 900 func (c *changefeed) handleBarrier(ctx context.Context, 901 cfInfo *model.ChangeFeedInfo, 902 cfStatus *model.ChangeFeedStatus, 903 barrier *schedulepb.BarrierWithMinTs, 904 ) error { 905 barrierTp, barrierTs := c.barriers.Min() 906 c.metricsChangefeedBarrierTsGauge.Set(float64(oracle.ExtractPhysical(barrierTs))) 907 908 // It means: 909 // 1. All data before the barrierTs was sent to downstream. 910 // 2. No more data after barrierTs was sent to downstream. 911 checkpointReachBarrier := barrierTs == cfStatus.CheckpointTs 912 if checkpointReachBarrier { 913 switch barrierTp { 914 case syncPointBarrier: 915 nextSyncPointTs := oracle.GoTimeToTS( 916 oracle.GetTimeFromTS(barrierTs). 917 Add(util.GetOrZero(cfInfo.Config.SyncPointInterval)), 918 ) 919 if err := c.ddlSink.emitSyncPoint(ctx, barrierTs); err != nil { 920 return errors.Trace(err) 921 } 922 c.barriers.Update(syncPointBarrier, nextSyncPointTs) 923 case finishBarrier: 924 c.feedStateManager.MarkFinished() 925 default: 926 log.Error("Unknown barrier type", zap.Int("barrierType", int(barrierTp))) 927 return cerror.ErrUnexpected.FastGenByArgs("Unknown barrier type") 928 } 929 } 930 931 // If there are other barriers less than ddl barrier, 932 // we should wait for them. 933 // Note: There may be some tableBarrierTs larger than otherBarrierTs, 934 // but we can ignore them because they will be handled in the processor. 935 if barrier.GlobalBarrierTs > barrierTs { 936 log.Debug("There are other barriers less than ddl barrier, wait for them", 937 zap.Uint64("otherBarrierTs", barrierTs), 938 zap.Uint64("globalBarrierTs", barrier.GlobalBarrierTs)) 939 barrier.GlobalBarrierTs = barrierTs 940 } 941 942 if barrier.MinTableBarrierTs > barrierTs { 943 log.Debug("There are other barriers less than min table barrier, wait for them", 944 zap.Uint64("otherBarrierTs", barrierTs), 945 zap.Uint64("minTableBarrierTs", barrier.GlobalBarrierTs)) 946 barrier.MinTableBarrierTs = barrierTs 947 } 948 949 return nil 950 } 951 952 func (c *changefeed) updateMetrics(currentTs int64, checkpointTs, resolvedTs model.Ts) { 953 phyCkpTs := oracle.ExtractPhysical(checkpointTs) 954 c.metricsChangefeedCheckpointTsGauge.Set(float64(phyCkpTs)) 955 956 checkpointLag := float64(currentTs-phyCkpTs) / 1e3 957 c.metricsChangefeedCheckpointTsLagGauge.Set(checkpointLag) 958 c.metricsChangefeedCheckpointLagDuration.Observe(checkpointLag) 959 960 phyRTs := oracle.ExtractPhysical(resolvedTs) 961 c.metricsChangefeedResolvedTsGauge.Set(float64(phyRTs)) 962 963 resolvedLag := float64(currentTs-phyRTs) / 1e3 964 c.metricsChangefeedResolvedTsLagGauge.Set(resolvedLag) 965 c.metricsChangefeedResolvedTsLagDuration.Observe(resolvedLag) 966 967 c.metricsCurrentPDTsGauge.Set(float64(currentTs)) 968 } 969 970 func (c *changefeed) Close(ctx context.Context) { 971 startTime := time.Now() 972 c.releaseResources(ctx) 973 974 costTime := time.Since(startTime) 975 if costTime > changefeedLogsWarnDuration { 976 log.Warn("changefeed close took too long", 977 zap.String("changefeed", c.id.ID), 978 zap.Duration("duration", costTime)) 979 } 980 changefeedCloseDuration.Observe(costTime.Seconds()) 981 } 982 983 // GetInfoProvider returns an InfoProvider if one is available. 984 func (c *changefeed) GetInfoProvider() scheduler.InfoProvider { 985 if provider, ok := c.scheduler.(scheduler.InfoProvider); ok { 986 return provider 987 } 988 return nil 989 } 990 991 // checkUpstream returns skip = true if the upstream is still in initializing phase, 992 // and returns an error if the upstream is unavailable. 993 func (c *changefeed) checkUpstream() (skip bool, err error) { 994 if err = c.upstream.Error(); err != nil { 995 return true, err 996 } 997 if c.upstream.IsClosed() { 998 log.Warn("upstream is closed", 999 zap.Uint64("upstreamID", c.upstream.ID), 1000 zap.String("namespace", c.id.Namespace), 1001 zap.String("changefeed", c.id.ID)) 1002 return true, cerror. 1003 WrapChangefeedUnretryableErr( 1004 cerror.ErrUpstreamClosed.GenWithStackByArgs()) 1005 } 1006 // upstream is still in initializing phase 1007 // skip this changefeed tick 1008 if !c.upstream.IsNormal() { 1009 return true, nil 1010 } 1011 return 1012 } 1013 1014 // tickDownstreamObserver checks whether needs to trigger tick of downstream 1015 // observer, if needed run it in an independent goroutine with 5s timeout. 1016 func (c *changefeed) tickDownstreamObserver(ctx context.Context) { 1017 if time.Since(c.observerLastTick.Load()) > downstreamObserverTickDuration { 1018 c.observerLastTick.Store(time.Now()) 1019 select { 1020 case <-ctx.Done(): 1021 return 1022 default: 1023 } 1024 go func() { 1025 cctx, cancel := context.WithTimeout(ctx, time.Second*5) 1026 defer cancel() 1027 if err := c.downstreamObserver.Tick(cctx); err != nil { 1028 // Prometheus is not deployed, it happens in non production env. 1029 noPrometheusMsg := fmt.Sprintf(":%d", errno.ErrPrometheusAddrIsNotSet) 1030 if strings.Contains(err.Error(), noPrometheusMsg) { 1031 return 1032 } 1033 log.Warn("backend observer tick error", zap.Error(err)) 1034 } 1035 }() 1036 } 1037 }