github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/owner.go (about) 1 // Copyright 2020 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 cdc 15 16 import ( 17 "context" 18 "fmt" 19 "io" 20 "math" 21 "sync" 22 "sync/atomic" 23 "time" 24 25 "github.com/pingcap/errors" 26 "github.com/pingcap/failpoint" 27 "github.com/pingcap/log" 28 "github.com/pingcap/ticdc/cdc/entry" 29 "github.com/pingcap/ticdc/cdc/kv" 30 "github.com/pingcap/ticdc/cdc/model" 31 "github.com/pingcap/ticdc/cdc/sink" 32 "github.com/pingcap/ticdc/pkg/config" 33 cdcContext "github.com/pingcap/ticdc/pkg/context" 34 "github.com/pingcap/ticdc/pkg/cyclic/mark" 35 cerror "github.com/pingcap/ticdc/pkg/errors" 36 "github.com/pingcap/ticdc/pkg/filter" 37 "github.com/pingcap/ticdc/pkg/notify" 38 "github.com/pingcap/ticdc/pkg/scheduler" 39 "github.com/pingcap/ticdc/pkg/util" 40 "github.com/pingcap/tidb/store/tikv/oracle" 41 pd "github.com/tikv/pd/client" 42 "go.etcd.io/etcd/clientv3" 43 "go.etcd.io/etcd/clientv3/concurrency" 44 "go.etcd.io/etcd/mvcc" 45 "go.uber.org/zap" 46 "golang.org/x/time/rate" 47 ) 48 49 type ownership struct { 50 lastTickTime time.Time 51 tickTime time.Duration 52 } 53 54 func newOwnership(tickTime time.Duration) ownership { 55 minTickTime := 5 * time.Second 56 if tickTime > minTickTime { 57 log.Panic("ownership counter must be incearsed every 5 seconds") 58 } 59 return ownership{ 60 tickTime: minTickTime, 61 } 62 } 63 64 func (o *ownership) inc() { 65 now := time.Now() 66 if now.Sub(o.lastTickTime) > o.tickTime { 67 // Keep the value of promtheus expression `rate(counter)` = 1 68 // Please also change alert rule in ticdc.rules.yml when change the expression value. 69 ownershipCounter.Add(float64(o.tickTime / time.Second)) 70 o.lastTickTime = now 71 } 72 } 73 74 type minGCSafePointCacheEntry struct { 75 ts model.Ts 76 lastUpdated time.Time 77 } 78 79 func (o *Owner) getMinGCSafePointCache(ctx context.Context) model.Ts { 80 if time.Now().After(o.minGCSafePointCache.lastUpdated.Add(MinGCSafePointCacheUpdateInterval)) { 81 physicalTs, logicalTs, err := o.pdClient.GetTS(ctx) 82 if err != nil { 83 log.Warn("Fail to update minGCSafePointCache.", zap.Error(err)) 84 return o.minGCSafePointCache.ts 85 } 86 o.minGCSafePointCache.ts = oracle.ComposeTS(physicalTs-(o.gcTTL*1000), logicalTs) 87 88 // o.pdGCSafePoint pd is the smallest gcSafePoint across all services. 89 // If tikv_gc_life_time > gcTTL, means that tikv_gc_safe_point < o.minGCSafePointCache.ts here. 90 // It also means that pd.pdGCSafePoint < o.minGCSafePointCache.ts here, we should use its value as the min value. 91 // This ensures that when tikv_gc_life_time > gcTTL , cdc will not advance the gcSafePoint. 92 if o.pdGCSafePoint < o.minGCSafePointCache.ts { 93 o.minGCSafePointCache.ts = o.pdGCSafePoint 94 } 95 96 o.minGCSafePointCache.lastUpdated = time.Now() 97 } 98 return o.minGCSafePointCache.ts 99 } 100 101 // Owner manages the cdc cluster 102 type Owner struct { 103 done chan struct{} 104 session *concurrency.Session 105 changeFeeds map[model.ChangeFeedID]*changeFeed 106 // failInitFeeds record changefeeds that meet error during initialization 107 failInitFeeds map[model.ChangeFeedID]struct{} 108 // stoppedFeeds record changefeeds that meet running error 109 stoppedFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus 110 rebalanceTigger map[model.ChangeFeedID]bool 111 rebalanceForAllChangefeed bool 112 manualScheduleCommand map[model.ChangeFeedID][]*model.MoveTableJob 113 rebalanceMu sync.Mutex 114 115 cfRWriter ChangeFeedRWriter 116 117 l sync.RWMutex 118 119 pdEndpoints []string 120 grpcPool kv.GrpcPool 121 pdClient pd.Client 122 etcdClient kv.CDCEtcdClient 123 124 captureLoaded int32 125 captures map[model.CaptureID]*model.CaptureInfo 126 127 adminJobs []model.AdminJob 128 adminJobsLock sync.Mutex 129 130 stepDown func(ctx context.Context) error 131 132 // gcTTL is the ttl of cdc gc safepoint ttl. 133 gcTTL int64 134 // last update gc safepoint time. zero time means has not updated or cleared 135 gcSafepointLastUpdate time.Time 136 // stores the ts obtained from PD and is updated every MinGCSafePointCacheUpdateInterval. 137 minGCSafePointCache minGCSafePointCacheEntry 138 // stores the actual gcSafePoint stored in pd 139 pdGCSafePoint model.Ts 140 // record last time that flushes all changefeeds' replication status 141 lastFlushChangefeeds time.Time 142 flushChangefeedInterval time.Duration 143 feedChangeNotifier *notify.Notifier 144 } 145 146 const ( 147 // CDCServiceSafePointID is the ID of CDC service in pd.UpdateServiceGCSafePoint. 148 CDCServiceSafePointID = "ticdc" 149 // GCSafepointUpdateInterval is the minimual interval that CDC can update gc safepoint 150 GCSafepointUpdateInterval = 2 * time.Second 151 // MinGCSafePointCacheUpdateInterval is the interval that update minGCSafePointCache 152 MinGCSafePointCacheUpdateInterval = time.Second * 2 153 ) 154 155 // NewOwner creates a new Owner instance 156 func NewOwner( 157 ctx context.Context, 158 pdClient pd.Client, 159 grpcPool kv.GrpcPool, 160 sess *concurrency.Session, 161 gcTTL int64, 162 flushChangefeedInterval time.Duration, 163 ) (*Owner, error) { 164 cli := kv.NewCDCEtcdClient(ctx, sess.Client()) 165 endpoints := sess.Client().Endpoints() 166 167 failpoint.Inject("ownerFlushIntervalInject", func(val failpoint.Value) { 168 flushChangefeedInterval = time.Millisecond * time.Duration(val.(int)) 169 }) 170 171 owner := &Owner{ 172 done: make(chan struct{}), 173 session: sess, 174 pdClient: pdClient, 175 grpcPool: grpcPool, 176 changeFeeds: make(map[model.ChangeFeedID]*changeFeed), 177 failInitFeeds: make(map[model.ChangeFeedID]struct{}), 178 stoppedFeeds: make(map[model.ChangeFeedID]*model.ChangeFeedStatus), 179 captures: make(map[model.CaptureID]*model.CaptureInfo), 180 rebalanceTigger: make(map[model.ChangeFeedID]bool), 181 manualScheduleCommand: make(map[model.ChangeFeedID][]*model.MoveTableJob), 182 pdEndpoints: endpoints, 183 cfRWriter: cli, 184 etcdClient: cli, 185 gcTTL: gcTTL, 186 flushChangefeedInterval: flushChangefeedInterval, 187 feedChangeNotifier: new(notify.Notifier), 188 } 189 190 return owner, nil 191 } 192 193 func (o *Owner) addCapture(_ context.Context, info *model.CaptureInfo) { 194 o.l.Lock() 195 o.captures[info.ID] = info 196 o.l.Unlock() 197 o.rebalanceMu.Lock() 198 o.rebalanceForAllChangefeed = true 199 o.rebalanceMu.Unlock() 200 } 201 202 // When a table is moved from one capture to another, the workflow is as follows 203 // 1. Owner deletes the table from the original capture (we call it capture-1), 204 // and adds an table operation record in the task status 205 // 2. The processor in capture-1 reads the operation record, and waits the table 206 // checkpoint ts reaches the boundary ts in operation, which often equals to 207 // the global resovled ts, larger the current checkpoint ts of this table. 208 // 3. After table checkpoint ts reaches boundary ts, capture-1 marks the table 209 // operation as finished. 210 // 4. Owner reads the finished mark and re-dispatches this table to another capture. 211 // 212 // When capture-1 crashes between step-2 and step-3, this function should be 213 // called to let owner re dispatch the table. Besides owner could also crash at 214 // the same time, in that case this function should also be called. In addtition, 215 // this function only handles move table job: 1) the add table job persists both 216 // table replicaInfo and operation, we can recover enough information from table 217 // replicaInfo; 2) if a table is deleted from a capture and that capture crashes, 218 // we just ignore this table. 219 func (o *Owner) rebuildTableFromOperations(cf *changeFeed, taskStatus *model.TaskStatus, startTs uint64) { 220 for tableID, op := range taskStatus.Operation { 221 if op.Delete && op.Flag&model.OperFlagMoveTable > 0 { 222 cf.orphanTables[tableID] = startTs 223 if job, ok := cf.moveTableJobs[tableID]; ok { 224 log.Info("remove outdated move table job", zap.Reflect("job", job), zap.Uint64("start-ts", startTs)) 225 delete(cf.moveTableJobs, tableID) 226 } 227 } 228 } 229 } 230 231 func (o *Owner) removeCapture(ctx context.Context, info *model.CaptureInfo) { 232 o.l.Lock() 233 defer o.l.Unlock() 234 235 delete(o.captures, info.ID) 236 237 for _, feed := range o.changeFeeds { 238 task, ok := feed.taskStatus[info.ID] 239 if !ok { 240 log.Warn("task status not found", zap.String("capture-id", info.ID), zap.String("changefeed", feed.id)) 241 continue 242 } 243 var startTs uint64 244 pos, ok := feed.taskPositions[info.ID] 245 if ok { 246 startTs = pos.CheckPointTs 247 } else { 248 log.Warn("task position not found, fallback to use changefeed checkpointts", 249 zap.String("capture-id", info.ID), zap.String("changefeed", feed.id)) 250 // maybe the processor hasn't added table yet, fallback to use the 251 // global checkpoint ts as the start ts of the table. 252 startTs = feed.status.CheckpointTs 253 } 254 255 for tableID, replicaInfo := range task.Tables { 256 feed.orphanTables[tableID] = startTs 257 if startTs < replicaInfo.StartTs { 258 log.Warn("table startTs not consistent", 259 zap.Uint64("table-start-ts", replicaInfo.StartTs), 260 zap.Uint64("checkpoint-ts", startTs), 261 zap.Reflect("status", feed.status)) 262 feed.orphanTables[tableID] = replicaInfo.StartTs 263 } 264 } 265 266 o.rebuildTableFromOperations(feed, task, startTs) 267 268 if err := o.etcdClient.LeaseGuardDeleteTaskStatus(ctx, feed.id, info.ID, o.session.Lease()); err != nil { 269 log.Warn("failed to delete task status", 270 zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) 271 } 272 if err := o.etcdClient.LeaseGuardDeleteTaskPosition(ctx, feed.id, info.ID, o.session.Lease()); err != nil { 273 log.Warn("failed to delete task position", 274 zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) 275 } 276 if err := o.etcdClient.LeaseGuardDeleteTaskWorkload(ctx, feed.id, info.ID, o.session.Lease()); err != nil { 277 log.Warn("failed to delete task workload", 278 zap.String("capture-id", info.ID), zap.String("changefeed", feed.id), zap.Error(err)) 279 } 280 ownerMaintainTableNumGauge.DeleteLabelValues(feed.id, info.AdvertiseAddr, maintainTableTypeTotal) 281 ownerMaintainTableNumGauge.DeleteLabelValues(feed.id, info.AdvertiseAddr, maintainTableTypeWip) 282 } 283 } 284 285 func (o *Owner) addOrphanTable(cid model.CaptureID, tableID model.TableID, startTs model.Ts) { 286 if cf, ok := o.changeFeeds[cid]; ok { 287 cf.orphanTables[tableID] = startTs 288 } else { 289 log.Warn("changefeed not found", zap.String("changefeed", cid)) 290 } 291 } 292 293 func (o *Owner) newChangeFeed( 294 ctx context.Context, 295 id model.ChangeFeedID, 296 processorsInfos model.ProcessorsInfos, 297 taskPositions map[string]*model.TaskPosition, 298 info *model.ChangeFeedInfo, 299 checkpointTs uint64) (cf *changeFeed, resultErr error) { 300 log.Info("Find new changefeed", zap.Stringer("info", info), 301 zap.String("changefeed", id), zap.Uint64("checkpoint ts", checkpointTs)) 302 if info.Config.CheckGCSafePoint { 303 err := util.CheckSafetyOfStartTs(ctx, o.pdClient, id, checkpointTs) 304 if err != nil { 305 return nil, errors.Trace(err) 306 } 307 } 308 failpoint.Inject("NewChangefeedNoRetryError", func() { 309 failpoint.Return(nil, cerror.ErrStartTsBeforeGC.GenWithStackByArgs(checkpointTs-300, checkpointTs)) 310 }) 311 312 failpoint.Inject("NewChangefeedRetryError", func() { 313 failpoint.Return(nil, errors.New("failpoint injected retriable error")) 314 }) 315 316 kvStore, err := util.KVStorageFromCtx(ctx) 317 if err != nil { 318 return nil, errors.Trace(err) 319 } 320 meta, err := kv.GetSnapshotMeta(kvStore, checkpointTs) 321 if err != nil { 322 return nil, errors.Trace(err) 323 } 324 schemaSnap, err := entry.NewSingleSchemaSnapshotFromMeta(meta, checkpointTs, info.Config.ForceReplicate) 325 if err != nil { 326 return nil, errors.Trace(err) 327 } 328 329 filter, err := filter.NewFilter(info.Config) 330 if err != nil { 331 return nil, errors.Trace(err) 332 } 333 334 ddlHandler := newDDLHandler(o.pdClient, o.grpcPool, kvStore, checkpointTs) 335 defer func() { 336 if resultErr != nil { 337 ddlHandler.Close() 338 } 339 }() 340 341 existingTables := make(map[model.TableID]model.Ts) 342 for captureID, taskStatus := range processorsInfos { 343 var checkpointTs uint64 344 if pos, exist := taskPositions[captureID]; exist { 345 checkpointTs = pos.CheckPointTs 346 } 347 for tableID, replicaInfo := range taskStatus.Tables { 348 if replicaInfo.StartTs > checkpointTs { 349 checkpointTs = replicaInfo.StartTs 350 } 351 existingTables[tableID] = checkpointTs 352 } 353 } 354 355 ctx, cancel := context.WithCancel(ctx) 356 cdcCtx := cdcContext.NewContext(ctx, &cdcContext.GlobalVars{}) 357 defer func() { 358 if resultErr != nil { 359 cancel() 360 } 361 }() 362 schemas := make(map[model.SchemaID]tableIDMap) 363 tables := make(map[model.TableID]model.TableName) 364 partitions := make(map[model.TableID][]int64) 365 orphanTables := make(map[model.TableID]model.Ts) 366 sinkTableInfo := make([]*model.SimpleTableInfo, len(schemaSnap.CloneTables())) 367 j := 0 368 for tid, table := range schemaSnap.CloneTables() { 369 j++ 370 if filter.ShouldIgnoreTable(table.Schema, table.Table) { 371 continue 372 } 373 if info.Config.Cyclic.IsEnabled() && mark.IsMarkTable(table.Schema, table.Table) { 374 // skip the mark table if cyclic is enabled 375 continue 376 } 377 378 tables[tid] = table 379 schema, ok := schemaSnap.SchemaByTableID(tid) 380 if !ok { 381 log.Warn("schema not found for table", zap.Int64("tid", tid)) 382 } else { 383 sid := schema.ID 384 if _, ok := schemas[sid]; !ok { 385 schemas[sid] = make(tableIDMap) 386 } 387 schemas[sid][tid] = struct{}{} 388 } 389 tblInfo, ok := schemaSnap.TableByID(tid) 390 if !ok { 391 log.Warn("table not found for table ID", zap.Int64("tid", tid)) 392 continue 393 } 394 if !tblInfo.IsEligible(info.Config.ForceReplicate) { 395 log.Warn("skip ineligible table", zap.Int64("tid", tid), zap.Stringer("table", table)) 396 continue 397 } 398 // `existingTables` are tables dispatched to a processor, however the 399 // capture that this processor belongs to could have crashed or exited. 400 // So we check this before task dispatching, but after the update of 401 // changefeed schema information. 402 if ts, ok := existingTables[tid]; ok { 403 log.Info("ignore known table", zap.Int64("tid", tid), zap.Stringer("table", table), zap.Uint64("ts", ts)) 404 continue 405 } 406 if pi := tblInfo.GetPartitionInfo(); pi != nil { 407 delete(partitions, tid) 408 for _, partition := range pi.Definitions { 409 id := partition.ID 410 partitions[tid] = append(partitions[tid], id) 411 if ts, ok := existingTables[id]; ok { 412 log.Info("ignore known table partition", zap.Int64("tid", tid), zap.Int64("partitionID", id), zap.Stringer("table", table), zap.Uint64("ts", ts)) 413 continue 414 } 415 orphanTables[id] = checkpointTs 416 } 417 } else { 418 orphanTables[tid] = checkpointTs 419 } 420 421 sinkTableInfo[j-1] = new(model.SimpleTableInfo) 422 sinkTableInfo[j-1].TableID = tid 423 sinkTableInfo[j-1].ColumnInfo = make([]*model.ColumnInfo, len(tblInfo.Cols())) 424 sinkTableInfo[j-1].Schema = table.Schema 425 sinkTableInfo[j-1].Table = table.Table 426 427 for i, colInfo := range tblInfo.Cols() { 428 sinkTableInfo[j-1].ColumnInfo[i] = new(model.ColumnInfo) 429 sinkTableInfo[j-1].ColumnInfo[i].FromTiColumnInfo(colInfo) 430 } 431 432 } 433 cdcCtx = cdcContext.WithChangefeedVars(cdcCtx, &cdcContext.ChangefeedVars{ 434 ID: id, 435 Info: info, 436 }) 437 errCh := make(chan error, defaultErrChSize) 438 cdcCtx = cdcContext.WithErrorHandler(cdcCtx, func(err error) error { 439 errCh <- errors.Trace(err) 440 return nil 441 }) 442 asyncSink, err := newAsyncSink(cdcCtx) 443 if err != nil { 444 return nil, errors.Trace(err) 445 } 446 defer func() { 447 if resultErr != nil && asyncSink != nil { 448 asyncSink.Close(ctx) 449 } 450 }() 451 go func() { 452 var err error 453 select { 454 case <-ctx.Done(): 455 case err = <-errCh: 456 cancel() 457 } 458 if err != nil && errors.Cause(err) != context.Canceled { 459 log.Error("error on running changefeed", zap.Error(err), zap.String("changefeed", id)) 460 // asyncSink error, stop the changefeed 461 var code string 462 if terror, ok := err.(*errors.Error); ok { 463 code = string(terror.RFCCode()) 464 } else { 465 code = string(cerror.ErrExecDDLFailed.RFCCode()) 466 } 467 _ = o.EnqueueJob(model.AdminJob{ 468 CfID: cf.id, 469 Type: model.AdminStop, 470 Error: &model.RunningError{ 471 Addr: util.CaptureAddrFromCtx(ctx), 472 Code: code, 473 Message: err.Error(), 474 }, 475 }) 476 } else { 477 log.Info("changefeed exited", zap.String("changfeed", id)) 478 } 479 }() 480 481 err = asyncSink.Initialize(cdcCtx, sinkTableInfo) 482 if err != nil { 483 log.Error("error on running owner", zap.Error(err)) 484 } 485 486 var syncpointStore sink.SyncpointStore 487 if info.SyncPointEnabled { 488 syncpointStore, err = sink.NewSyncpointStore(ctx, id, info.SinkURI) 489 if err != nil { 490 return nil, errors.Trace(err) 491 } 492 } 493 494 cf = &changeFeed{ 495 info: info, 496 id: id, 497 ddlHandler: ddlHandler, 498 schema: schemaSnap, 499 schemas: schemas, 500 tables: tables, 501 partitions: partitions, 502 orphanTables: orphanTables, 503 toCleanTables: make(map[model.TableID]model.Ts), 504 status: &model.ChangeFeedStatus{ 505 ResolvedTs: 0, 506 CheckpointTs: checkpointTs, 507 }, 508 appliedCheckpointTs: checkpointTs, 509 scheduler: scheduler.NewScheduler(info.Config.Scheduler.Tp), 510 ddlState: model.ChangeFeedSyncDML, 511 ddlExecutedTs: checkpointTs, 512 targetTs: info.GetTargetTs(), 513 ddlTs: 0, 514 updateResolvedTs: true, 515 startTimer: make(chan bool), 516 syncpointStore: syncpointStore, 517 syncCancel: nil, 518 taskStatus: processorsInfos, 519 taskPositions: taskPositions, 520 etcdCli: o.etcdClient, 521 leaseID: o.session.Lease(), 522 filter: filter, 523 sink: asyncSink, 524 cyclicEnabled: info.Config.Cyclic.IsEnabled(), 525 lastRebalanceTime: time.Now(), 526 cdcCtx: cdcCtx, 527 cancel: cancel, 528 } 529 return cf, nil 530 } 531 532 // This is a compatibility hack between v4.0.0 and v4.0.1 533 // This function will try to decode the task status, if that throw a unmarshal error, 534 // it will remove the invalid task status 535 func (o *Owner) checkAndCleanTasksInfo(ctx context.Context) error { 536 _, details, err := o.cfRWriter.GetChangeFeeds(ctx) 537 if err != nil { 538 return err 539 } 540 cleaned := false 541 for changefeedID := range details { 542 _, err := o.cfRWriter.GetAllTaskStatus(ctx, changefeedID) 543 if err != nil { 544 if cerror.ErrDecodeFailed.NotEqual(err) { 545 return errors.Trace(err) 546 } 547 err := o.cfRWriter.LeaseGuardRemoveAllTaskStatus(ctx, changefeedID, o.session.Lease()) 548 if err != nil { 549 return errors.Trace(err) 550 } 551 cleaned = true 552 } 553 } 554 if cleaned { 555 log.Warn("the task status is outdated, clean them") 556 } 557 return nil 558 } 559 560 func (o *Owner) loadChangeFeeds(ctx context.Context) error { 561 _, details, err := o.cfRWriter.GetChangeFeeds(ctx) 562 if err != nil { 563 return err 564 } 565 errorFeeds := make(map[model.ChangeFeedID]*model.RunningError) 566 for changeFeedID, cfInfoRawValue := range details { 567 taskStatus, err := o.cfRWriter.GetAllTaskStatus(ctx, changeFeedID) 568 if err != nil { 569 return err 570 } 571 taskPositions, err := o.cfRWriter.GetAllTaskPositions(ctx, changeFeedID) 572 if err != nil { 573 return err 574 } 575 if cf, exist := o.changeFeeds[changeFeedID]; exist { 576 cf.updateProcessorInfos(taskStatus, taskPositions) 577 for _, pos := range taskPositions { 578 // TODO: only record error of one capture, 579 // is it necessary to record all captures' error 580 if pos.Error != nil { 581 errorFeeds[changeFeedID] = pos.Error 582 break 583 } 584 } 585 continue 586 } 587 588 // we find a new changefeed, init changefeed here. 589 cfInfo := &model.ChangeFeedInfo{} 590 err = cfInfo.Unmarshal(cfInfoRawValue.Value) 591 if err != nil { 592 return err 593 } 594 if cfInfo.State == model.StateFailed { 595 if _, ok := o.failInitFeeds[changeFeedID]; ok { 596 continue 597 } 598 log.Warn("changefeed is not in normal state", zap.String("changefeed", changeFeedID)) 599 o.failInitFeeds[changeFeedID] = struct{}{} 600 continue 601 } 602 if _, ok := o.failInitFeeds[changeFeedID]; ok { 603 log.Info("changefeed recovered from failure", zap.String("changefeed", changeFeedID)) 604 delete(o.failInitFeeds, changeFeedID) 605 } 606 needSave, canInit := cfInfo.CheckErrorHistory() 607 if needSave { 608 err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) 609 if err != nil { 610 return err 611 } 612 } 613 if !canInit { 614 // avoid too many logs here 615 if time.Now().Unix()%60 == 0 { 616 log.Warn("changefeed fails reach rate limit, try to initialize it later", zap.Int64s("history", cfInfo.ErrorHis)) 617 } 618 continue 619 } 620 err = cfInfo.VerifyAndFix() 621 if err != nil { 622 return err 623 } 624 625 status, _, err := o.cfRWriter.GetChangeFeedStatus(ctx, changeFeedID) 626 if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { 627 return err 628 } 629 if status != nil && status.AdminJobType.IsStopState() { 630 if status.AdminJobType == model.AdminStop { 631 if _, ok := o.stoppedFeeds[changeFeedID]; !ok { 632 o.stoppedFeeds[changeFeedID] = status 633 } 634 } 635 continue 636 } 637 638 // remaining task status means some processors are not exited, wait until 639 // all these statuses cleaned. If the capture of pending processor loses 640 // etcd session, the cleanUpStaleTasks will clean these statuses later. 641 allMetadataCleaned := true 642 allTaskStatus, err := o.etcdClient.GetAllTaskStatus(ctx, changeFeedID) 643 if err != nil { 644 return err 645 } 646 for _, taskStatus := range allTaskStatus { 647 if taskStatus.AdminJobType == model.AdminStop || taskStatus.AdminJobType == model.AdminRemove { 648 log.Info("stale task status is not deleted, wait metadata cleaned to create new changefeed", 649 zap.Reflect("task status", taskStatus), zap.String("changefeed", changeFeedID)) 650 allMetadataCleaned = false 651 break 652 } 653 } 654 if !allMetadataCleaned { 655 continue 656 } 657 658 checkpointTs := cfInfo.GetCheckpointTs(status) 659 660 newCf, err := o.newChangeFeed(ctx, changeFeedID, taskStatus, taskPositions, cfInfo, checkpointTs) 661 if err != nil { 662 cfInfo.Error = &model.RunningError{ 663 Addr: util.CaptureAddrFromCtx(ctx), 664 Code: "CDC-owner-1001", 665 Message: err.Error(), 666 } 667 cfInfo.ErrorHis = append(cfInfo.ErrorHis, time.Now().UnixNano()/1e6) 668 669 if cerror.ChangefeedFastFailError(err) { 670 log.Error("create changefeed with fast fail error, mark changefeed as failed", 671 zap.Error(err), zap.String("changefeed", changeFeedID)) 672 cfInfo.State = model.StateFailed 673 err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) 674 if err != nil { 675 return err 676 } 677 continue 678 } 679 680 err2 := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, changeFeedID, o.session.Lease()) 681 if err2 != nil { 682 return err2 683 } 684 // changefeed error has been recorded in etcd, log error here and 685 // don't need to return an error. 686 log.Warn("create changefeed failed, retry later", 687 zap.String("changefeed", changeFeedID), zap.Error(err)) 688 continue 689 } 690 691 if newCf.info.SyncPointEnabled { 692 log.Info("syncpoint is on, creating the sync table") 693 // create the sync table 694 err := newCf.syncpointStore.CreateSynctable(ctx) 695 if err != nil { 696 return err 697 } 698 newCf.startSyncPointTicker(ctx, newCf.info.SyncPointInterval) 699 } else { 700 log.Info("syncpoint is off") 701 } 702 703 o.changeFeeds[changeFeedID] = newCf 704 delete(o.stoppedFeeds, changeFeedID) 705 } 706 o.adminJobsLock.Lock() 707 for cfID, err := range errorFeeds { 708 job := model.AdminJob{ 709 CfID: cfID, 710 Type: model.AdminStop, 711 Error: err, 712 } 713 o.adminJobs = append(o.adminJobs, job) 714 } 715 o.adminJobsLock.Unlock() 716 return nil 717 } 718 719 func (o *Owner) balanceTables(ctx context.Context) error { 720 rebalanceForAllChangefeed := false 721 o.rebalanceMu.Lock() 722 if o.rebalanceForAllChangefeed { 723 rebalanceForAllChangefeed = true 724 o.rebalanceForAllChangefeed = false 725 } 726 o.rebalanceMu.Unlock() 727 for id, changefeed := range o.changeFeeds { 728 rebalanceNow := false 729 var scheduleCommands []*model.MoveTableJob 730 o.rebalanceMu.Lock() 731 if r, exist := o.rebalanceTigger[id]; exist { 732 rebalanceNow = r 733 delete(o.rebalanceTigger, id) 734 } 735 if rebalanceForAllChangefeed { 736 rebalanceNow = true 737 } 738 if c, exist := o.manualScheduleCommand[id]; exist { 739 scheduleCommands = c 740 delete(o.manualScheduleCommand, id) 741 } 742 o.rebalanceMu.Unlock() 743 err := changefeed.tryBalance(ctx, o.captures, rebalanceNow, scheduleCommands) 744 if err != nil { 745 return errors.Trace(err) 746 } 747 } 748 return nil 749 } 750 751 func (o *Owner) flushChangeFeedInfos(ctx context.Context) error { 752 // no running or stopped changefeed, clear gc safepoint. 753 if len(o.changeFeeds) == 0 && len(o.stoppedFeeds) == 0 { 754 if !o.gcSafepointLastUpdate.IsZero() { 755 log.Info("clean service safe point", zap.String("service-id", CDCServiceSafePointID)) 756 _, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, 0, 0) 757 if err != nil { 758 log.Warn("failed to update service safe point", zap.Error(err)) 759 } else { 760 o.gcSafepointLastUpdate = time.Time{} 761 } 762 } 763 return nil 764 } 765 766 staleChangeFeeds := make(map[model.ChangeFeedID]*model.ChangeFeedStatus, len(o.changeFeeds)) 767 gcSafePoint := uint64(math.MaxUint64) 768 769 // get the lower bound of gcSafePoint 770 minGCSafePoint := o.getMinGCSafePointCache(ctx) 771 772 if len(o.changeFeeds) > 0 { 773 snapshot := make(map[model.ChangeFeedID]*model.ChangeFeedStatus, len(o.changeFeeds)) 774 for id, changefeed := range o.changeFeeds { 775 snapshot[id] = changefeed.status 776 if changefeed.status.CheckpointTs < gcSafePoint { 777 gcSafePoint = changefeed.status.CheckpointTs 778 } 779 // 1. If changefeed's appliedCheckpoinTs <= minGCSafePoint, it means this changefeed is stagnant. 780 // They are collected into this map, and then handleStaleChangeFeed() is called to deal with these stagnant changefeed. 781 // A changefeed will not enter the map twice, because in run(), 782 // handleAdminJob() will always be executed before flushChangeFeedInfos(), 783 // ensuring that the previous changefeed in staleChangeFeeds has been stopped and removed from o.changeFeeds. 784 // 2. We need the `<=` check here is because when a changefeed is stagnant, its checkpointTs will be updated to pd, 785 // and it would be the minimum gcSafePoint across all services. 786 // So as described above(line 92) minGCSafePoint = gcSafePoint = CheckpointTs would happens. 787 // In this case, if we check `<` here , this changefeed will not be put into staleChangeFeeds, and its checkpoints will be updated to pd again and again. 788 // This will cause the cdc's gcSafePoint never advance. 789 // If we check `<=` here, when we encounter the changefeed again, we will put it into staleChangeFeeds. 790 if changefeed.status.CheckpointTs <= minGCSafePoint { 791 staleChangeFeeds[id] = changefeed.status 792 } 793 794 phyTs := oracle.ExtractPhysical(changefeed.status.CheckpointTs) 795 changefeedCheckpointTsGauge.WithLabelValues(id).Set(float64(phyTs)) 796 // It is more accurate to get tso from PD, but in most cases we have 797 // deployed NTP service, a little bias is acceptable here. 798 changefeedCheckpointTsLagGauge.WithLabelValues(id).Set(float64(oracle.GetPhysical(time.Now())-phyTs) / 1e3) 799 } 800 if time.Since(o.lastFlushChangefeeds) > o.flushChangefeedInterval { 801 err := o.cfRWriter.LeaseGuardPutAllChangeFeedStatus(ctx, snapshot, o.session.Lease()) 802 if err != nil { 803 return errors.Trace(err) 804 } 805 for id, changefeedStatus := range snapshot { 806 o.changeFeeds[id].appliedCheckpointTs = changefeedStatus.CheckpointTs 807 } 808 o.lastFlushChangefeeds = time.Now() 809 } 810 } 811 812 for _, status := range o.stoppedFeeds { 813 // If a stopped changefeed's CheckpoinTs <= minGCSafePoint, means this changefeed is stagnant. 814 // It should never be resumed. This part of the logic is in newChangeFeed() 815 // So here we can skip it. 816 if status.CheckpointTs <= minGCSafePoint { 817 continue 818 } 819 820 if status.CheckpointTs < gcSafePoint { 821 gcSafePoint = status.CheckpointTs 822 } 823 } 824 825 // handle stagnant changefeed collected above 826 err := o.handleStaleChangeFeed(ctx, staleChangeFeeds, minGCSafePoint) 827 if err != nil { 828 log.Warn("failed to handleStaleChangeFeed ", zap.Error(err)) 829 } 830 831 if time.Since(o.gcSafepointLastUpdate) > GCSafepointUpdateInterval { 832 actual, err := o.pdClient.UpdateServiceGCSafePoint(ctx, CDCServiceSafePointID, o.gcTTL, gcSafePoint) 833 if err != nil { 834 sinceLastUpdate := time.Since(o.gcSafepointLastUpdate) 835 log.Warn("failed to update service safe point", zap.Error(err), 836 zap.Duration("since-last-update", sinceLastUpdate)) 837 // We do not throw an error unless updating GC safepoint has been failing for more than gcTTL. 838 if sinceLastUpdate >= time.Second*time.Duration(o.gcTTL) { 839 return cerror.ErrUpdateServiceSafepointFailed.Wrap(err) 840 } 841 } else { 842 o.pdGCSafePoint = actual 843 o.gcSafepointLastUpdate = time.Now() 844 } 845 846 failpoint.Inject("InjectActualGCSafePoint", func(val failpoint.Value) { 847 actual = uint64(val.(int)) 848 }) 849 850 if actual > gcSafePoint { 851 // UpdateServiceGCSafePoint has failed. 852 log.Warn("updating an outdated service safe point", zap.Uint64("checkpoint-ts", gcSafePoint), zap.Uint64("actual-safepoint", actual)) 853 854 for cfID, cf := range o.changeFeeds { 855 if cf.status.CheckpointTs < actual { 856 runningError := &model.RunningError{ 857 Addr: util.CaptureAddrFromCtx(ctx), 858 Code: "CDC-owner-1001", 859 Message: cerror.ErrServiceSafepointLost.GenWithStackByArgs(actual).Error(), 860 } 861 862 err := o.EnqueueJob(model.AdminJob{ 863 CfID: cfID, 864 Type: model.AdminStop, 865 Error: runningError, 866 }) 867 if err != nil { 868 return errors.Trace(err) 869 } 870 } 871 } 872 } 873 } 874 return nil 875 } 876 877 // calcResolvedTs call calcResolvedTs of every changefeeds 878 func (o *Owner) calcResolvedTs(ctx context.Context) error { 879 for id, cf := range o.changeFeeds { 880 if err := cf.calcResolvedTs(); err != nil { 881 log.Error("fail to calculate checkpoint ts, so it will be stopped", zap.String("changefeed", cf.id), zap.Error(err)) 882 // error may cause by sink.EmitCheckpointTs`, just stop the changefeed at the moment 883 // todo: make the method mentioned above more robust. 884 var code string 885 if rfcCode, ok := cerror.RFCCode(err); ok { 886 code = string(rfcCode) 887 } else { 888 code = string(cerror.ErrOwnerUnknown.RFCCode()) 889 } 890 891 job := model.AdminJob{ 892 CfID: id, 893 Type: model.AdminStop, 894 Error: &model.RunningError{ 895 Addr: util.CaptureAddrFromCtx(ctx), 896 Code: code, 897 Message: err.Error(), 898 }, 899 } 900 901 if err := o.EnqueueJob(job); err != nil { 902 return errors.Trace(err) 903 } 904 } 905 } 906 return nil 907 } 908 909 // handleDDL call handleDDL of every changefeeds 910 func (o *Owner) handleDDL(ctx context.Context) error { 911 for _, cf := range o.changeFeeds { 912 err := cf.handleDDL() 913 if err != nil { 914 var code string 915 if terror, ok := err.(*errors.Error); ok { 916 code = string(terror.RFCCode()) 917 } else { 918 code = string(cerror.ErrExecDDLFailed.RFCCode()) 919 } 920 err = o.EnqueueJob(model.AdminJob{ 921 CfID: cf.id, 922 Type: model.AdminStop, 923 Error: &model.RunningError{ 924 Addr: util.CaptureAddrFromCtx(ctx), 925 Code: code, 926 Message: err.Error(), 927 }, 928 }) 929 if err != nil { 930 return errors.Trace(err) 931 } 932 } 933 } 934 return nil 935 } 936 937 // handleSyncPoint call handleSyncPoint of every changefeeds 938 func (o *Owner) handleSyncPoint(ctx context.Context) error { 939 for _, cf := range o.changeFeeds { 940 if err := cf.handleSyncPoint(ctx); err != nil { 941 return errors.Trace(err) 942 } 943 } 944 return nil 945 } 946 947 // dispatchJob dispatches job to processors 948 // Note job type in this function contains pause, remove and finish 949 func (o *Owner) dispatchJob(ctx context.Context, job model.AdminJob) error { 950 cf, ok := o.changeFeeds[job.CfID] 951 if !ok { 952 return cerror.ErrOwnerChangefeedNotFound.GenWithStackByArgs(job.CfID) 953 } 954 for captureID := range cf.taskStatus { 955 newStatus, _, err := cf.etcdCli.LeaseGuardAtomicPutTaskStatus( 956 ctx, cf.id, captureID, o.session.Lease(), 957 func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) { 958 taskStatus.AdminJobType = job.Type 959 return true, nil 960 }, 961 ) 962 if err != nil { 963 return errors.Trace(err) 964 } 965 cf.taskStatus[captureID] = newStatus.Clone() 966 } 967 // record admin job in changefeed status 968 cf.status.AdminJobType = job.Type 969 infos := map[model.ChangeFeedID]*model.ChangeFeedStatus{job.CfID: cf.status} 970 err := o.cfRWriter.LeaseGuardPutAllChangeFeedStatus(ctx, infos, o.session.Lease()) 971 if err != nil { 972 return errors.Trace(err) 973 } 974 cf.Close() 975 // Only need to process stoppedFeeds with `AdminStop` command here. 976 // For `AdminResume`, we remove stopped feed in changefeed initialization phase. 977 // For `AdminRemove`, we need to update stoppedFeeds when removing a stopped changefeed. 978 if job.Type == model.AdminStop { 979 log.Debug("put changefeed into stoppedFeeds queue", zap.String("changefeed", job.CfID)) 980 o.stoppedFeeds[job.CfID] = cf.status 981 } 982 for captureID := range cf.taskStatus { 983 capture, ok := o.captures[captureID] 984 if !ok { 985 log.Warn("capture not found", zap.String("capture-id", captureID)) 986 continue 987 } 988 ownerMaintainTableNumGauge.DeleteLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeTotal) 989 ownerMaintainTableNumGauge.DeleteLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeWip) 990 } 991 delete(o.changeFeeds, job.CfID) 992 changefeedCheckpointTsGauge.DeleteLabelValues(cf.id) 993 changefeedCheckpointTsLagGauge.DeleteLabelValues(cf.id) 994 return nil 995 } 996 997 func (o *Owner) collectChangefeedInfo(ctx context.Context, cid model.ChangeFeedID) ( 998 cf *changeFeed, 999 status *model.ChangeFeedStatus, 1000 feedState model.FeedState, 1001 err error, 1002 ) { 1003 var ok bool 1004 cf, ok = o.changeFeeds[cid] 1005 if ok { 1006 return cf, cf.status, cf.info.State, nil 1007 } 1008 feedState = model.StateNormal 1009 1010 var cfInfo *model.ChangeFeedInfo 1011 cfInfo, err = o.etcdClient.GetChangeFeedInfo(ctx, cid) 1012 if err != nil && cerror.ErrChangeFeedNotExists.NotEqual(err) { 1013 return 1014 } 1015 1016 status, _, err = o.etcdClient.GetChangeFeedStatus(ctx, cid) 1017 if err != nil { 1018 if cerror.ErrChangeFeedNotExists.Equal(err) { 1019 // Only changefeed info exists and error field is not nil means 1020 // the changefeed has met error, mark it as failed. 1021 if cfInfo != nil && cfInfo.Error != nil { 1022 feedState = model.StateFailed 1023 } 1024 } 1025 return 1026 } 1027 switch status.AdminJobType { 1028 case model.AdminNone, model.AdminResume: 1029 if cfInfo != nil && cfInfo.Error != nil { 1030 feedState = model.StateFailed 1031 } 1032 case model.AdminStop: 1033 feedState = model.StateStopped 1034 case model.AdminRemove: 1035 feedState = model.StateRemoved 1036 case model.AdminFinish: 1037 feedState = model.StateFinished 1038 } 1039 return 1040 } 1041 1042 func (o *Owner) checkClusterHealth(_ context.Context) error { 1043 // check whether a changefeed has finished by comparing checkpoint-ts and target-ts 1044 for _, cf := range o.changeFeeds { 1045 if cf.status.CheckpointTs == cf.info.GetTargetTs() { 1046 log.Info("changefeed replication finished", zap.String("changefeed", cf.id), zap.Uint64("checkpointTs", cf.status.CheckpointTs)) 1047 err := o.EnqueueJob(model.AdminJob{ 1048 CfID: cf.id, 1049 Type: model.AdminFinish, 1050 }) 1051 if err != nil { 1052 return err 1053 } 1054 } 1055 } 1056 for _, cf := range o.changeFeeds { 1057 for captureID, pinfo := range cf.taskStatus { 1058 capture, ok := o.captures[captureID] 1059 if !ok { 1060 log.Warn("capture not found", zap.String("capture-id", captureID)) 1061 continue 1062 } 1063 ownerMaintainTableNumGauge.WithLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeTotal).Set(float64(len(pinfo.Tables))) 1064 ownerMaintainTableNumGauge.WithLabelValues(cf.id, capture.AdvertiseAddr, maintainTableTypeWip).Set(float64(len(pinfo.Operation))) 1065 } 1066 } 1067 // TODO: check processor normal exited 1068 return nil 1069 } 1070 1071 func (o *Owner) handleAdminJob(ctx context.Context) error { 1072 removeIdx := 0 1073 o.adminJobsLock.Lock() 1074 defer func() { 1075 o.adminJobs = o.adminJobs[removeIdx:] 1076 o.adminJobsLock.Unlock() 1077 }() 1078 for i, job := range o.adminJobs { 1079 log.Info("handle admin job", zap.String("changefeed", job.CfID), zap.Stringer("type", job.Type)) 1080 removeIdx = i + 1 1081 1082 cf, status, feedState, err := o.collectChangefeedInfo(ctx, job.CfID) 1083 if err != nil { 1084 if cerror.ErrChangeFeedNotExists.NotEqual(err) { 1085 return err 1086 } 1087 if feedState == model.StateFailed && job.Type == model.AdminRemove { 1088 // changefeed in failed state, but changefeed status has not 1089 // been created yet. Try to remove changefeed info only. 1090 err := o.etcdClient.LeaseGuardDeleteChangeFeedInfo(ctx, job.CfID, o.session.Lease()) 1091 if err != nil { 1092 return errors.Trace(err) 1093 } 1094 } else { 1095 log.Warn("invalid admin job, changefeed status not found", zap.String("changefeed", job.CfID)) 1096 } 1097 continue 1098 } 1099 switch job.Type { 1100 case model.AdminStop: 1101 switch feedState { 1102 case model.StateStopped: 1103 log.Info("changefeed has been stopped, pause command will do nothing") 1104 continue 1105 case model.StateRemoved: 1106 log.Info("changefeed has been removed, pause command will do nothing") 1107 continue 1108 case model.StateFinished: 1109 log.Info("changefeed has finished, pause command will do nothing") 1110 continue 1111 } 1112 if cf == nil { 1113 log.Warn("invalid admin job, changefeed not found", zap.String("changefeed", job.CfID)) 1114 continue 1115 } 1116 1117 cf.info.AdminJobType = model.AdminStop 1118 cf.info.Error = job.Error 1119 if job.Error != nil { 1120 cf.info.ErrorHis = append(cf.info.ErrorHis, time.Now().UnixNano()/1e6) 1121 } 1122 1123 err := o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cf.info, job.CfID, o.session.Lease()) 1124 if err != nil { 1125 return errors.Trace(err) 1126 } 1127 err = o.dispatchJob(ctx, job) 1128 if err != nil { 1129 return errors.Trace(err) 1130 } 1131 cf.stopSyncPointTicker() 1132 case model.AdminRemove, model.AdminFinish: 1133 if cf != nil { 1134 cf.stopSyncPointTicker() 1135 err := o.dispatchJob(ctx, job) 1136 if err != nil { 1137 return errors.Trace(err) 1138 } 1139 } else { 1140 switch feedState { 1141 case model.StateRemoved, model.StateFinished: 1142 // remove a removed or finished changefeed 1143 if job.Opts != nil && job.Opts.ForceRemove { 1144 err := o.etcdClient.LeaseGuardRemoveChangeFeedStatus(ctx, job.CfID, o.session.Lease()) 1145 if err != nil { 1146 return errors.Trace(err) 1147 } 1148 } else { 1149 log.Info("changefeed has been removed or finished, remove command will do nothing") 1150 } 1151 continue 1152 case model.StateStopped, model.StateFailed: 1153 // remove a paused or failed changefeed 1154 status.AdminJobType = model.AdminRemove 1155 err = o.etcdClient.LeaseGuardPutChangeFeedStatus(ctx, job.CfID, status, o.session.Lease()) 1156 if err != nil { 1157 return errors.Trace(err) 1158 } 1159 delete(o.stoppedFeeds, job.CfID) 1160 default: 1161 return cerror.ErrChangefeedAbnormalState.GenWithStackByArgs(feedState, status) 1162 } 1163 } 1164 // remove changefeed info 1165 err := o.etcdClient.DeleteChangeFeedInfo(ctx, job.CfID) 1166 if err != nil { 1167 return errors.Trace(err) 1168 } 1169 if job.Opts != nil && job.Opts.ForceRemove { 1170 // if `ForceRemove` is enabled, remove all information related to this changefeed 1171 err := o.etcdClient.LeaseGuardRemoveChangeFeedStatus(ctx, job.CfID, o.session.Lease()) 1172 if err != nil { 1173 return errors.Trace(err) 1174 } 1175 } else { 1176 // set ttl to changefeed status 1177 err = o.etcdClient.SetChangeFeedStatusTTL(ctx, job.CfID, 24*3600 /*24 hours*/) 1178 if err != nil { 1179 return errors.Trace(err) 1180 } 1181 } 1182 case model.AdminResume: 1183 // resume changefeed must read checkpoint from ChangeFeedStatus 1184 if cerror.ErrChangeFeedNotExists.Equal(err) { 1185 log.Warn("invalid admin job, changefeed not found", zap.String("changefeed", job.CfID)) 1186 continue 1187 } 1188 if feedState == model.StateRemoved || feedState == model.StateFinished { 1189 log.Info("changefeed has been removed or finished, cannot be resumed anymore") 1190 continue 1191 } 1192 cfInfo, err := o.etcdClient.GetChangeFeedInfo(ctx, job.CfID) 1193 if err != nil { 1194 return errors.Trace(err) 1195 } 1196 1197 // set admin job in changefeed status to tell owner resume changefeed 1198 status.AdminJobType = model.AdminResume 1199 err = o.etcdClient.LeaseGuardPutChangeFeedStatus(ctx, job.CfID, status, o.session.Lease()) 1200 if err != nil { 1201 return errors.Trace(err) 1202 } 1203 1204 // set admin job in changefeed cfInfo to trigger each capture's changefeed list watch event 1205 cfInfo.AdminJobType = model.AdminResume 1206 // clear last running error 1207 cfInfo.State = model.StateNormal 1208 cfInfo.Error = nil 1209 err = o.etcdClient.LeaseGuardSaveChangeFeedInfo(ctx, cfInfo, job.CfID, o.session.Lease()) 1210 if err != nil { 1211 return errors.Trace(err) 1212 } 1213 if config.NewReplicaImpl { 1214 // remove all positions because the old positions may be include an error 1215 err = o.etcdClient.RemoveAllTaskPositions(ctx, job.CfID) 1216 if err != nil { 1217 return errors.Trace(err) 1218 } 1219 } 1220 } 1221 // TODO: we need a better admin job workflow. Supposing uses create 1222 // multiple admin jobs to a specific changefeed at the same time, such 1223 // as pause -> resume -> pause, should the one job handler waits for 1224 // the previous job finished? However it is difficult to distinguish 1225 // whether a job is totally finished in some cases, for example when 1226 // resuming a changefeed, seems we should mark the job finished if all 1227 // processors have started. Currently the owner only processes one 1228 // admin job in each tick loop as a workaround. 1229 break 1230 } 1231 return nil 1232 } 1233 1234 func (o *Owner) throne(ctx context.Context) error { 1235 // Start a routine to keep watching on the liveness of 1236 // captures. 1237 o.startCaptureWatcher(ctx) 1238 return nil 1239 } 1240 1241 // Close stops a running owner 1242 func (o *Owner) Close(ctx context.Context, stepDown func(ctx context.Context) error) { 1243 // stepDown is called after exiting the main loop by the owner, it is useful 1244 // to clean up some resource, like dropping the leader key. 1245 o.stepDown = stepDown 1246 1247 // Close and Run should be in separated goroutines 1248 // A channel is used here to synchronize the steps. 1249 1250 // Single the Run function to exit 1251 select { 1252 case o.done <- struct{}{}: 1253 case <-ctx.Done(): 1254 } 1255 1256 // Wait until it exited 1257 select { 1258 case <-o.done: 1259 case <-ctx.Done(): 1260 } 1261 } 1262 1263 // Run the owner 1264 // TODO avoid this tick style, this means we get `tickTime` latency here. 1265 func (o *Owner) Run(ctx context.Context, tickTime time.Duration) error { 1266 failpoint.Inject("owner-run-with-error", func() { 1267 failpoint.Return(errors.New("owner run with injected error")) 1268 }) 1269 1270 ctx, cancel := context.WithCancel(ctx) 1271 defer cancel() 1272 1273 go func() { 1274 if err := o.watchCampaignKey(ctx); err != nil { 1275 cancel() 1276 } 1277 }() 1278 1279 if err := o.throne(ctx); err != nil { 1280 return err 1281 } 1282 1283 ctx1, cancel1 := context.WithCancel(ctx) 1284 defer cancel1() 1285 feedChangeReceiver, err := o.feedChangeNotifier.NewReceiver(tickTime) 1286 if err != nil { 1287 return err 1288 } 1289 defer feedChangeReceiver.Stop() 1290 o.watchFeedChange(ctx1) 1291 1292 ownership := newOwnership(tickTime) 1293 loop: 1294 for { 1295 select { 1296 case <-o.done: 1297 close(o.done) 1298 break loop 1299 case <-ctx.Done(): 1300 // FIXME: cancel the context doesn't ensure all resources are destructed, is it reasonable? 1301 // Anyway we just break loop here to ensure the following destruction. 1302 err = ctx.Err() 1303 break loop 1304 case <-feedChangeReceiver.C: 1305 ownership.inc() 1306 } 1307 1308 err = o.run(ctx) 1309 if err != nil { 1310 switch errors.Cause(err) { 1311 case context.DeadlineExceeded: 1312 // context timeout means the o.run doesn't finish in a safe owner 1313 // lease cycle, it is safe to retry. If the lease is revoked, 1314 // another run loop will detect it. 1315 continue loop 1316 case context.Canceled: 1317 default: 1318 log.Error("owner exited with error", zap.Error(err)) 1319 } 1320 break loop 1321 } 1322 } 1323 for _, cf := range o.changeFeeds { 1324 cf.Close() 1325 changefeedCheckpointTsGauge.DeleteLabelValues(cf.id) 1326 changefeedCheckpointTsLagGauge.DeleteLabelValues(cf.id) 1327 } 1328 if o.stepDown != nil { 1329 if err := o.stepDown(ctx); err != nil { 1330 return err 1331 } 1332 } 1333 1334 return err 1335 } 1336 1337 // watchCampaignKey watches the aliveness of campaign owner key in etcd 1338 func (o *Owner) watchCampaignKey(ctx context.Context) error { 1339 key := fmt.Sprintf("%s/%x", kv.CaptureOwnerKey, o.session.Lease()) 1340 restart: 1341 resp, err := o.etcdClient.Client.Get(ctx, key) 1342 if err != nil { 1343 return cerror.WrapError(cerror.ErrPDEtcdAPIError, err) 1344 } 1345 if resp.Count == 0 { 1346 return cerror.ErrOwnerCampaignKeyDeleted.GenWithStackByArgs() 1347 } 1348 // watch the key change from the next revision relatived to the current 1349 wch := o.etcdClient.Client.Watch(ctx, key, clientv3.WithRev(resp.Header.Revision+1)) 1350 for resp := range wch { 1351 err := resp.Err() 1352 if err != nil { 1353 if err != mvcc.ErrCompacted { 1354 log.Error("watch owner campaign key failed, restart the watcher", zap.Error(err)) 1355 } 1356 goto restart 1357 } 1358 for _, ev := range resp.Events { 1359 if ev.Type == clientv3.EventTypeDelete { 1360 log.Warn("owner campaign key deleted", zap.String("key", key)) 1361 return cerror.ErrOwnerCampaignKeyDeleted.GenWithStackByArgs() 1362 } 1363 } 1364 } 1365 return nil 1366 } 1367 1368 func (o *Owner) watchFeedChange(ctx context.Context) { 1369 go func() { 1370 for { 1371 select { 1372 case <-ctx.Done(): 1373 return 1374 default: 1375 } 1376 cctx, cancel := context.WithCancel(ctx) 1377 wch := o.etcdClient.Client.Watch(cctx, kv.TaskPositionKeyPrefix, clientv3.WithFilterDelete(), clientv3.WithPrefix()) 1378 1379 for resp := range wch { 1380 if resp.Err() != nil { 1381 log.Error("position watcher restarted with error", zap.Error(resp.Err())) 1382 break 1383 } 1384 1385 // TODO: because the main loop has many serial steps, it is hard to do a partial update without change 1386 // majority logical. For now just to wakeup the main loop ASAP to reduce latency, the efficiency of etcd 1387 // operations should be resolved in future release. 1388 1389 o.feedChangeNotifier.Notify() 1390 } 1391 cancel() 1392 } 1393 }() 1394 } 1395 1396 func (o *Owner) run(ctx context.Context) error { 1397 // captureLoaded == 0 means capture information is not built, owner can't 1398 // run normal jobs now. 1399 if atomic.LoadInt32(&o.captureLoaded) == int32(0) { 1400 return nil 1401 } 1402 1403 o.l.Lock() 1404 defer o.l.Unlock() 1405 1406 var err error 1407 1408 err = o.cleanUpStaleTasks(ctx) 1409 if err != nil { 1410 return errors.Trace(err) 1411 } 1412 1413 err = o.loadChangeFeeds(ctx) 1414 if err != nil { 1415 return errors.Trace(err) 1416 } 1417 1418 err = o.balanceTables(ctx) 1419 if err != nil { 1420 return errors.Trace(err) 1421 } 1422 1423 err = o.handleDDL(ctx) 1424 if err != nil { 1425 return errors.Trace(err) 1426 } 1427 1428 err = o.handleSyncPoint(ctx) 1429 if err != nil { 1430 return errors.Trace(err) 1431 } 1432 1433 err = o.handleAdminJob(ctx) 1434 if err != nil { 1435 return errors.Trace(err) 1436 } 1437 1438 err = o.calcResolvedTs(ctx) 1439 if err != nil { 1440 return errors.Trace(err) 1441 } 1442 1443 // It is better for flushChangeFeedInfos to follow calcResolvedTs immediately, 1444 // because operations such as handleDDL and rebalancing rely on proper progress of the checkpoint in Etcd. 1445 err = o.flushChangeFeedInfos(ctx) 1446 if err != nil { 1447 return errors.Trace(err) 1448 } 1449 1450 err = o.checkClusterHealth(ctx) 1451 if err != nil { 1452 return errors.Trace(err) 1453 } 1454 1455 return nil 1456 } 1457 1458 // EnqueueJob adds an admin job 1459 func (o *Owner) EnqueueJob(job model.AdminJob) error { 1460 switch job.Type { 1461 case model.AdminResume, model.AdminRemove, model.AdminStop, model.AdminFinish: 1462 default: 1463 return cerror.ErrInvalidAdminJobType.GenWithStackByArgs(job.Type) 1464 } 1465 o.adminJobsLock.Lock() 1466 o.adminJobs = append(o.adminJobs, job) 1467 o.adminJobsLock.Unlock() 1468 return nil 1469 } 1470 1471 // TriggerRebalance triggers the rebalance in the specified changefeed 1472 func (o *Owner) TriggerRebalance(changefeedID model.ChangeFeedID) { 1473 o.rebalanceMu.Lock() 1474 defer o.rebalanceMu.Unlock() 1475 o.rebalanceTigger[changefeedID] = true 1476 // TODO(leoppro) throw an error if the changefeed is not exist 1477 } 1478 1479 // ManualSchedule moves the table from a capture to another capture 1480 func (o *Owner) ManualSchedule(changefeedID model.ChangeFeedID, to model.CaptureID, tableID model.TableID) { 1481 o.rebalanceMu.Lock() 1482 defer o.rebalanceMu.Unlock() 1483 o.manualScheduleCommand[changefeedID] = append(o.manualScheduleCommand[changefeedID], &model.MoveTableJob{ 1484 To: to, 1485 TableID: tableID, 1486 }) 1487 } 1488 1489 func (o *Owner) writeDebugInfo(w io.Writer) { 1490 fmt.Fprintf(w, "** active changefeeds **:\n") 1491 for _, info := range o.changeFeeds { 1492 fmt.Fprintf(w, "%s\n", info) 1493 } 1494 fmt.Fprintf(w, "** stopped changefeeds **:\n") 1495 for _, feedStatus := range o.stoppedFeeds { 1496 fmt.Fprintf(w, "%+v\n", *feedStatus) 1497 } 1498 fmt.Fprintf(w, "\n** captures **:\n") 1499 for _, capture := range o.captures { 1500 fmt.Fprintf(w, "%+v\n", *capture) 1501 } 1502 } 1503 1504 // cleanUpStaleTasks cleans up the task status which does not associated 1505 // with an active processor. This function is not thread safe. 1506 // 1507 // When a new owner is elected, it does not know the events occurs before, like 1508 // processor deletion. In this case, the new owner should check if the task 1509 // status is stale because of the processor deletion. 1510 func (o *Owner) cleanUpStaleTasks(ctx context.Context) error { 1511 _, changefeeds, err := o.etcdClient.GetChangeFeeds(ctx) 1512 if err != nil { 1513 return errors.Trace(err) 1514 } 1515 for changeFeedID := range changefeeds { 1516 statuses, err := o.etcdClient.GetAllTaskStatus(ctx, changeFeedID) 1517 if err != nil { 1518 return errors.Trace(err) 1519 } 1520 positions, err := o.etcdClient.GetAllTaskPositions(ctx, changeFeedID) 1521 if err != nil { 1522 return errors.Trace(err) 1523 } 1524 workloads, err := o.etcdClient.GetAllTaskWorkloads(ctx, changeFeedID) 1525 if err != nil { 1526 return errors.Trace(err) 1527 } 1528 // in most cases statuses and positions have the same keys, or positions 1529 // are more than statuses, as we always delete task status first. 1530 captureIDs := make(map[string]struct{}, len(statuses)) 1531 for captureID := range statuses { 1532 captureIDs[captureID] = struct{}{} 1533 } 1534 for captureID := range positions { 1535 captureIDs[captureID] = struct{}{} 1536 } 1537 for captureID := range workloads { 1538 captureIDs[captureID] = struct{}{} 1539 } 1540 1541 log.Debug("cleanUpStaleTasks", 1542 zap.Reflect("statuses", statuses), 1543 zap.Reflect("positions", positions), 1544 zap.Reflect("workloads", workloads)) 1545 1546 for captureID := range captureIDs { 1547 if _, ok := o.captures[captureID]; !ok { 1548 status, ok1 := statuses[captureID] 1549 if ok1 { 1550 pos, taskPosFound := positions[captureID] 1551 if !taskPosFound { 1552 log.Warn("task position not found, fallback to use original start ts", 1553 zap.String("capture", captureID), 1554 zap.String("changefeed", changeFeedID), 1555 zap.Reflect("task status", status), 1556 ) 1557 } 1558 for tableID, replicaInfo := range status.Tables { 1559 startTs := replicaInfo.StartTs 1560 if taskPosFound { 1561 if startTs < pos.CheckPointTs { 1562 startTs = pos.CheckPointTs 1563 } 1564 } 1565 o.addOrphanTable(changeFeedID, tableID, startTs) 1566 } 1567 if cf, ok := o.changeFeeds[changeFeedID]; ok { 1568 o.rebuildTableFromOperations(cf, status, cf.status.CheckpointTs) 1569 } 1570 } 1571 1572 if err := o.etcdClient.LeaseGuardDeleteTaskStatus(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { 1573 return errors.Trace(err) 1574 } 1575 if err := o.etcdClient.LeaseGuardDeleteTaskPosition(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { 1576 return errors.Trace(err) 1577 } 1578 if err := o.etcdClient.LeaseGuardDeleteTaskWorkload(ctx, changeFeedID, captureID, o.session.Lease()); err != nil { 1579 return errors.Trace(err) 1580 } 1581 log.Info("cleanup stale task", zap.String("capture-id", captureID), zap.String("changefeed", changeFeedID)) 1582 } 1583 } 1584 } 1585 return nil 1586 } 1587 1588 func (o *Owner) watchCapture(ctx context.Context) error { 1589 ctx = clientv3.WithRequireLeader(ctx) 1590 1591 failpoint.Inject("sleep-before-watch-capture", nil) 1592 1593 // When an owner just starts, changefeed information is not updated at once. 1594 // Supposing a crashed capture should be removed now, the owner will miss deleting 1595 // task status and task position if changefeed information is not loaded. 1596 // If the task positions and status decode failed, remove them. 1597 if err := o.checkAndCleanTasksInfo(ctx); err != nil { 1598 return errors.Trace(err) 1599 } 1600 o.l.Lock() 1601 if err := o.loadChangeFeeds(ctx); err != nil { 1602 o.l.Unlock() 1603 return errors.Trace(err) 1604 } 1605 o.l.Unlock() 1606 1607 rev, captureList, err := o.etcdClient.GetCaptures(ctx) 1608 if err != nil { 1609 return errors.Trace(err) 1610 } 1611 captures := make(map[model.CaptureID]*model.CaptureInfo) 1612 for _, c := range captureList { 1613 captures[c.ID] = c 1614 } 1615 // before watching, rebuild events according to 1616 // the existed captures. This is necessary because 1617 // the etcd events may be compacted. 1618 if err := o.rebuildCaptureEvents(ctx, captures); err != nil { 1619 return errors.Trace(err) 1620 } 1621 1622 log.Info("monitoring captures", 1623 zap.String("key", kv.CaptureInfoKeyPrefix), 1624 zap.Int64("rev", rev)) 1625 ch := o.etcdClient.Client.Watch(ctx, kv.CaptureInfoKeyPrefix, 1626 clientv3.WithPrefix(), 1627 clientv3.WithRev(rev+1), 1628 clientv3.WithPrevKV()) 1629 1630 for resp := range ch { 1631 err := resp.Err() 1632 failpoint.Inject("restart-capture-watch", func() { 1633 err = mvcc.ErrCompacted 1634 }) 1635 if err != nil { 1636 return cerror.WrapError(cerror.ErrOwnerEtcdWatch, resp.Err()) 1637 } 1638 for _, ev := range resp.Events { 1639 c := &model.CaptureInfo{} 1640 switch ev.Type { 1641 case clientv3.EventTypeDelete: 1642 if err := c.Unmarshal(ev.PrevKv.Value); err != nil { 1643 return errors.Trace(err) 1644 } 1645 log.Info("delete capture", 1646 zap.String("capture-id", c.ID), 1647 zap.String("capture", c.AdvertiseAddr)) 1648 o.removeCapture(ctx, c) 1649 case clientv3.EventTypePut: 1650 if !ev.IsCreate() { 1651 continue 1652 } 1653 if err := c.Unmarshal(ev.Kv.Value); err != nil { 1654 return errors.Trace(err) 1655 } 1656 log.Info("add capture", 1657 zap.String("capture-id", c.ID), 1658 zap.String("capture", c.AdvertiseAddr)) 1659 o.addCapture(ctx, c) 1660 } 1661 } 1662 } 1663 return nil 1664 } 1665 1666 func (o *Owner) rebuildCaptureEvents(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error { 1667 for _, c := range captures { 1668 o.addCapture(ctx, c) 1669 } 1670 for _, c := range o.captures { 1671 if _, ok := captures[c.ID]; !ok { 1672 o.removeCapture(ctx, c) 1673 } 1674 } 1675 // captureLoaded is used to check whether the owner can execute cleanup stale tasks job. 1676 // Because at the very beginning of a new owner, it doesn't have capture information in 1677 // memory, cleanup stale tasks could have a false positive (where positive means owner 1678 // should cleanup the stale task of a specific capture). After the first time of capture 1679 // rebuild, even the etcd compaction and watch capture is rerun, we don't need to check 1680 // captureLoaded anymore because existing tasks must belong to a capture which is still 1681 // maintained in owner's memory. 1682 atomic.StoreInt32(&o.captureLoaded, 1) 1683 1684 // clean up stale tasks each time before watch capture event starts, 1685 // for two reasons: 1686 // 1. when a new owner is elected, it must clean up stale task status and positions. 1687 // 2. when error happens in owner's capture event watch, the owner just resets 1688 // the watch loop, with the following two steps: 1689 // 1) load all captures from PD, having a revision for data 1690 // 2) start a new watch from revision in step1 1691 // the step-2 may meet an error such as ErrCompacted, and we will continue 1692 // from step-1, however other capture may crash just after step-2 returns 1693 // and before step-1 starts, the longer time gap between step-2 to step-1, 1694 // missing a crashed capture is more likely to happen. 1695 o.l.Lock() 1696 defer o.l.Unlock() 1697 return errors.Trace(o.cleanUpStaleTasks(ctx)) 1698 } 1699 1700 func (o *Owner) startCaptureWatcher(ctx context.Context) { 1701 log.Info("start to watch captures") 1702 go func() { 1703 rl := rate.NewLimiter(0.05, 2) 1704 for { 1705 err := rl.Wait(ctx) 1706 if err != nil { 1707 if errors.Cause(err) == context.Canceled { 1708 return 1709 } 1710 log.Error("capture watcher wait limit token error", zap.Error(err)) 1711 return 1712 } 1713 if err := o.watchCapture(ctx); err != nil { 1714 // When the watching routine returns, the error must not 1715 // be nil, it may be caused by a temporary error or a context 1716 // error(ctx.Err()) 1717 if ctx.Err() != nil { 1718 if errors.Cause(ctx.Err()) != context.Canceled { 1719 // The context error indicates the termination of the owner 1720 log.Error("watch capture failed", zap.Error(ctx.Err())) 1721 } else { 1722 log.Info("watch capture exited") 1723 } 1724 return 1725 } 1726 log.Warn("watch capture returned", zap.Error(err)) 1727 // Otherwise, a temporary error occurred(ErrCompact), 1728 // restart the watching routine. 1729 } 1730 } 1731 }() 1732 } 1733 1734 // handle the StaleChangeFeed 1735 // By setting the AdminJob type to AdminStop and the Error code to indicate that the changefeed is stagnant. 1736 func (o *Owner) handleStaleChangeFeed(ctx context.Context, staleChangeFeeds map[model.ChangeFeedID]*model.ChangeFeedStatus, minGCSafePoint uint64) error { 1737 for id, status := range staleChangeFeeds { 1738 err := cerror.ErrSnapshotLostByGC.GenWithStackByArgs(status.CheckpointTs, minGCSafePoint) 1739 log.Warn("changefeed checkpoint is lagging too much, so it will be stopped.", zap.String("changefeed", id), zap.Error(err)) 1740 runningError := &model.RunningError{ 1741 Addr: util.CaptureAddrFromCtx(ctx), 1742 Code: string(cerror.ErrSnapshotLostByGC.RFCCode()), // changefeed is stagnant 1743 Message: err.Error(), 1744 } 1745 1746 err = o.EnqueueJob(model.AdminJob{ 1747 CfID: id, 1748 Type: model.AdminStop, 1749 Error: runningError, 1750 }) 1751 if err != nil { 1752 return errors.Trace(err) 1753 } 1754 delete(staleChangeFeeds, id) 1755 } 1756 return nil 1757 }