github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/processor/processor.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 processor 15 16 import ( 17 "context" 18 "fmt" 19 "io" 20 "math" 21 "strconv" 22 "sync" 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 tablepipeline "github.com/pingcap/ticdc/cdc/processor/pipeline" 32 "github.com/pingcap/ticdc/cdc/puller" 33 "github.com/pingcap/ticdc/cdc/sink" 34 cdcContext "github.com/pingcap/ticdc/pkg/context" 35 "github.com/pingcap/ticdc/pkg/cyclic/mark" 36 cerror "github.com/pingcap/ticdc/pkg/errors" 37 "github.com/pingcap/ticdc/pkg/filter" 38 "github.com/pingcap/ticdc/pkg/orchestrator" 39 "github.com/pingcap/ticdc/pkg/regionspan" 40 "github.com/pingcap/ticdc/pkg/retry" 41 "github.com/pingcap/ticdc/pkg/util" 42 "github.com/pingcap/tidb/store/tikv/oracle" 43 "github.com/prometheus/client_golang/prometheus" 44 "go.uber.org/zap" 45 ) 46 47 const ( 48 schemaStorageGCLag = time.Minute * 20 49 50 backoffBaseDelayInMs = 5 51 maxTries = 3 52 ) 53 54 type processor struct { 55 changefeedID model.ChangeFeedID 56 captureInfo *model.CaptureInfo 57 changefeed *model.ChangefeedReactorState 58 59 tables map[model.TableID]tablepipeline.TablePipeline 60 61 schemaStorage entry.SchemaStorage 62 filter *filter.Filter 63 mounter entry.Mounter 64 sinkManager *sink.Manager 65 66 initialized bool 67 errCh chan error 68 cancel context.CancelFunc 69 wg sync.WaitGroup 70 71 lazyInit func(ctx cdcContext.Context) error 72 createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) 73 74 metricResolvedTsGauge prometheus.Gauge 75 metricResolvedTsLagGauge prometheus.Gauge 76 metricCheckpointTsGauge prometheus.Gauge 77 metricCheckpointTsLagGauge prometheus.Gauge 78 metricSyncTableNumGauge prometheus.Gauge 79 metricProcessorErrorCounter prometheus.Counter 80 } 81 82 // newProcessor creates a new processor 83 func newProcessor(ctx cdcContext.Context) *processor { 84 changefeedID := ctx.ChangefeedVars().ID 85 advertiseAddr := ctx.GlobalVars().CaptureInfo.AdvertiseAddr 86 p := &processor{ 87 tables: make(map[model.TableID]tablepipeline.TablePipeline), 88 errCh: make(chan error, 1), 89 changefeedID: changefeedID, 90 captureInfo: ctx.GlobalVars().CaptureInfo, 91 cancel: func() {}, 92 93 metricResolvedTsGauge: resolvedTsGauge.WithLabelValues(changefeedID, advertiseAddr), 94 metricResolvedTsLagGauge: resolvedTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), 95 metricCheckpointTsGauge: checkpointTsGauge.WithLabelValues(changefeedID, advertiseAddr), 96 metricCheckpointTsLagGauge: checkpointTsLagGauge.WithLabelValues(changefeedID, advertiseAddr), 97 metricSyncTableNumGauge: syncTableNumGauge.WithLabelValues(changefeedID, advertiseAddr), 98 metricProcessorErrorCounter: processorErrorCounter.WithLabelValues(changefeedID, advertiseAddr), 99 } 100 p.createTablePipeline = p.createTablePipelineImpl 101 p.lazyInit = p.lazyInitImpl 102 return p 103 } 104 105 func newProcessor4Test(ctx cdcContext.Context, 106 createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error), 107 ) *processor { 108 p := newProcessor(ctx) 109 p.lazyInit = func(ctx cdcContext.Context) error { return nil } 110 p.createTablePipeline = createTablePipeline 111 return p 112 } 113 114 // Tick implements the `orchestrator.State` interface 115 // the `state` parameter is sent by the etcd worker, the `state` must be a snapshot of KVs in etcd 116 // The main logic of processor is in this function, including the calculation of many kinds of ts, maintain table pipeline, error handling, etc. 117 func (p *processor) Tick(ctx cdcContext.Context, state *model.ChangefeedReactorState) (orchestrator.ReactorState, error) { 118 p.changefeed = state 119 state.CheckCaptureAlive(ctx.GlobalVars().CaptureInfo.ID) 120 ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ 121 ID: state.ID, 122 Info: state.Info, 123 }) 124 _, err := p.tick(ctx, state) 125 if err == nil { 126 return state, nil 127 } 128 cause := errors.Cause(err) 129 if cause == context.Canceled || cerror.ErrAdminStopProcessor.Equal(cause) || cerror.ErrReactorFinished.Equal(cause) { 130 log.Info("processor exited", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) 131 return state, cerror.ErrReactorFinished.GenWithStackByArgs() 132 } 133 p.metricProcessorErrorCounter.Inc() 134 // record error information in etcd 135 var code string 136 if rfcCode, ok := cerror.RFCCode(err); ok { 137 code = string(rfcCode) 138 } else { 139 code = string(cerror.ErrProcessorUnknown.RFCCode()) 140 } 141 state.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 142 if position == nil { 143 position = &model.TaskPosition{} 144 } 145 position.Error = &model.RunningError{ 146 Addr: ctx.GlobalVars().CaptureInfo.AdvertiseAddr, 147 Code: code, 148 Message: err.Error(), 149 } 150 return position, true, nil 151 }) 152 log.Error("run processor failed", 153 cdcContext.ZapFieldChangefeed(ctx), 154 cdcContext.ZapFieldCapture(ctx), 155 zap.Error(err)) 156 return state, cerror.ErrReactorFinished.GenWithStackByArgs() 157 } 158 159 func (p *processor) tick(ctx cdcContext.Context, state *model.ChangefeedReactorState) (nextState orchestrator.ReactorState, err error) { 160 p.changefeed = state 161 if !p.checkChangefeedNormal() { 162 return nil, cerror.ErrAdminStopProcessor.GenWithStackByArgs() 163 } 164 if skip := p.checkPosition(); skip { 165 return p.changefeed, nil 166 } 167 if err := p.handleErrorCh(ctx); err != nil { 168 return nil, errors.Trace(err) 169 } 170 if err := p.lazyInit(ctx); err != nil { 171 return nil, errors.Trace(err) 172 } 173 if err := p.handleTableOperation(ctx); err != nil { 174 return nil, errors.Trace(err) 175 } 176 if err := p.checkTablesNum(ctx); err != nil { 177 return nil, errors.Trace(err) 178 } 179 p.handlePosition() 180 p.pushResolvedTs2Table() 181 p.handleWorkload() 182 p.doGCSchemaStorage() 183 return p.changefeed, nil 184 } 185 186 // checkChangefeedNormal checks if the changefeed is runnable. 187 func (p *processor) checkChangefeedNormal() bool { 188 // check the state in this tick, make sure that the admin job type of the changefeed is not stopped 189 if p.changefeed.Info.AdminJobType.IsStopState() || p.changefeed.Status.AdminJobType.IsStopState() { 190 return false 191 } 192 // add a patch to check the changefeed is runnable when applying the patches in the etcd worker. 193 p.changefeed.CheckChangefeedNormal() 194 return true 195 } 196 197 // checkPosition create a new task position, and put it into the etcd state. 198 // task position maybe be not exist only when the processor is running first time. 199 func (p *processor) checkPosition() (skipThisTick bool) { 200 if p.changefeed.TaskPositions[p.captureInfo.ID] != nil { 201 return false 202 } 203 if p.initialized { 204 log.Warn("position is nil, maybe position info is removed unexpected", zap.Any("state", p.changefeed)) 205 } 206 checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) 207 p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 208 if position == nil { 209 return &model.TaskPosition{ 210 CheckPointTs: checkpointTs, 211 ResolvedTs: checkpointTs, 212 }, true, nil 213 } 214 return position, false, nil 215 }) 216 return true 217 } 218 219 // lazyInitImpl create Filter, SchemaStorage, Mounter instances at the first tick. 220 func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { 221 if p.initialized { 222 return nil 223 } 224 ctx, cancel := cdcContext.WithCancel(ctx) 225 p.cancel = cancel 226 227 errCh := make(chan error, 16) 228 p.wg.Add(1) 229 go func() { 230 defer p.wg.Done() 231 // there are some other objects need errCh, such as sink and sink manager 232 // but we can't ensure that all the producer of errCh are non-blocking 233 // It's very tricky that create a goroutine to receive the local errCh 234 // TODO(leoppro): we should using `pkg/cdcContext.Context` instead of standard cdcContext and handle error by `pkg/cdcContext.Context.Throw` 235 for { 236 select { 237 case <-ctx.Done(): 238 close(errCh) 239 return 240 case err := <-errCh: 241 if err == nil { 242 return 243 } 244 p.sendError(err) 245 } 246 } 247 }() 248 249 var err error 250 p.filter, err = filter.NewFilter(p.changefeed.Info.Config) 251 if err != nil { 252 return errors.Trace(err) 253 } 254 255 p.schemaStorage, err = p.createAndDriveSchemaStorage(ctx) 256 if err != nil { 257 return errors.Trace(err) 258 } 259 260 stdCtx := util.PutChangefeedIDInCtx(ctx, p.changefeed.ID) 261 262 p.mounter = entry.NewMounter(p.schemaStorage, p.changefeed.Info.Config.Mounter.WorkerNum, p.changefeed.Info.Config.EnableOldValue) 263 p.wg.Add(1) 264 go func() { 265 defer p.wg.Done() 266 p.sendError(p.mounter.Run(stdCtx)) 267 }() 268 269 opts := make(map[string]string, len(p.changefeed.Info.Opts)+2) 270 for k, v := range p.changefeed.Info.Opts { 271 opts[k] = v 272 } 273 274 // TODO(neil) find a better way to let sink know cyclic is enabled. 275 if p.changefeed.Info.Config.Cyclic.IsEnabled() { 276 cyclicCfg, err := p.changefeed.Info.Config.Cyclic.Marshal() 277 if err != nil { 278 return errors.Trace(err) 279 } 280 opts[mark.OptCyclicConfig] = cyclicCfg 281 } 282 opts[sink.OptChangefeedID] = p.changefeed.ID 283 opts[sink.OptCaptureAddr] = ctx.GlobalVars().CaptureInfo.AdvertiseAddr 284 s, err := sink.NewSink(stdCtx, p.changefeed.ID, p.changefeed.Info.SinkURI, p.filter, p.changefeed.Info.Config, opts, errCh) 285 if err != nil { 286 return errors.Trace(err) 287 } 288 checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) 289 p.sinkManager = sink.NewManager(stdCtx, s, errCh, checkpointTs) 290 p.initialized = true 291 log.Info("run processor", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) 292 return nil 293 } 294 295 // handleErrorCh listen the error channel and throw the error if it is not expected. 296 func (p *processor) handleErrorCh(ctx cdcContext.Context) error { 297 var err error 298 select { 299 case err = <-p.errCh: 300 default: 301 return nil 302 } 303 cause := errors.Cause(err) 304 if cause != nil && cause != context.Canceled && cerror.ErrAdminStopProcessor.NotEqual(cause) { 305 log.Error("error on running processor", 306 cdcContext.ZapFieldCapture(ctx), 307 cdcContext.ZapFieldChangefeed(ctx), 308 zap.Error(err)) 309 return err 310 } 311 log.Info("processor exited", cdcContext.ZapFieldCapture(ctx), cdcContext.ZapFieldChangefeed(ctx)) 312 return cerror.ErrReactorFinished 313 } 314 315 // handleTableOperation handles the operation of `TaskStatus`(add table operation and remove table operation) 316 func (p *processor) handleTableOperation(ctx cdcContext.Context) error { 317 patchOperation := func(tableID model.TableID, fn func(operation *model.TableOperation) error) { 318 p.changefeed.PatchTaskStatus(p.captureInfo.ID, func(status *model.TaskStatus) (*model.TaskStatus, bool, error) { 319 if status == nil || status.Operation == nil { 320 log.Error("Operation not found, may be remove by other patch", zap.Int64("tableID", tableID), zap.Any("status", status)) 321 return nil, false, cerror.ErrTaskStatusNotExists.GenWithStackByArgs() 322 } 323 opt := status.Operation[tableID] 324 if opt == nil { 325 log.Error("Operation not found, may be remove by other patch", zap.Int64("tableID", tableID), zap.Any("status", status)) 326 return nil, false, cerror.ErrTaskStatusNotExists.GenWithStackByArgs() 327 } 328 if err := fn(opt); err != nil { 329 return nil, false, errors.Trace(err) 330 } 331 return status, true, nil 332 }) 333 } 334 taskStatus := p.changefeed.TaskStatuses[p.captureInfo.ID] 335 for tableID, opt := range taskStatus.Operation { 336 if opt.TableApplied() { 337 continue 338 } 339 globalCheckpointTs := p.changefeed.Status.CheckpointTs 340 if opt.Delete { 341 table, exist := p.tables[tableID] 342 if !exist { 343 log.Warn("table which will be deleted is not found", 344 cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) 345 patchOperation(tableID, func(operation *model.TableOperation) error { 346 operation.Status = model.OperFinished 347 operation.Done = true 348 return nil 349 }) 350 continue 351 } 352 switch opt.Status { 353 case model.OperDispatched: 354 if opt.BoundaryTs < globalCheckpointTs { 355 log.Warn("the BoundaryTs of remove table operation is smaller than global checkpoint ts", zap.Uint64("globalCheckpointTs", globalCheckpointTs), zap.Any("operation", opt)) 356 } 357 if !table.AsyncStop(opt.BoundaryTs) { 358 // We use a Debug log because it is conceivable for the pipeline to block for a legitimate reason, 359 // and we do not want to alarm the user. 360 log.Debug("AsyncStop has failed, possible due to a full pipeline", 361 zap.Uint64("checkpointTs", table.CheckpointTs()), zap.Int64("tableID", tableID)) 362 continue 363 } 364 patchOperation(tableID, func(operation *model.TableOperation) error { 365 operation.Status = model.OperProcessed 366 return nil 367 }) 368 case model.OperProcessed: 369 if table.Status() != tablepipeline.TableStatusStopped { 370 log.Debug("the table is still not stopped", zap.Uint64("checkpointTs", table.CheckpointTs()), zap.Int64("tableID", tableID)) 371 continue 372 } 373 patchOperation(tableID, func(operation *model.TableOperation) error { 374 operation.BoundaryTs = table.CheckpointTs() 375 operation.Status = model.OperFinished 376 operation.Done = true 377 return nil 378 }) 379 table.Cancel() 380 table.Wait() 381 delete(p.tables, tableID) 382 log.Debug("Operation done signal received", 383 cdcContext.ZapFieldChangefeed(ctx), 384 zap.Int64("tableID", tableID), 385 zap.Reflect("operation", opt)) 386 default: 387 log.Panic("unreachable") 388 } 389 } else { 390 switch opt.Status { 391 case model.OperDispatched: 392 replicaInfo, exist := taskStatus.Tables[tableID] 393 if !exist { 394 return cerror.ErrProcessorTableNotFound.GenWithStack("replicaInfo of table(%d)", tableID) 395 } 396 if replicaInfo.StartTs != opt.BoundaryTs { 397 log.Warn("the startTs and BoundaryTs of add table operation should be always equaled", zap.Any("replicaInfo", replicaInfo)) 398 } 399 err := p.addTable(ctx, tableID, replicaInfo) 400 if err != nil { 401 return errors.Trace(err) 402 } 403 patchOperation(tableID, func(operation *model.TableOperation) error { 404 operation.Status = model.OperProcessed 405 return nil 406 }) 407 case model.OperProcessed: 408 table, exist := p.tables[tableID] 409 if !exist { 410 log.Warn("table which was added is not found", 411 cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) 412 patchOperation(tableID, func(operation *model.TableOperation) error { 413 operation.Status = model.OperDispatched 414 return nil 415 }) 416 continue 417 } 418 localResolvedTs := p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs 419 globalResolvedTs := p.changefeed.Status.ResolvedTs 420 if table.ResolvedTs() >= localResolvedTs && localResolvedTs >= globalResolvedTs { 421 patchOperation(tableID, func(operation *model.TableOperation) error { 422 operation.Status = model.OperFinished 423 operation.Done = true 424 return nil 425 }) 426 log.Debug("Operation done signal received", 427 cdcContext.ZapFieldChangefeed(ctx), 428 zap.Int64("tableID", tableID), 429 zap.Reflect("operation", opt)) 430 } 431 default: 432 log.Panic("unreachable") 433 } 434 } 435 } 436 return nil 437 } 438 439 func (p *processor) createAndDriveSchemaStorage(ctx cdcContext.Context) (entry.SchemaStorage, error) { 440 kvStorage := ctx.GlobalVars().KVStorage 441 ddlspans := []regionspan.Span{regionspan.GetDDLSpan(), regionspan.GetAddIndexDDLSpan()} 442 checkpointTs := p.changefeed.Info.GetCheckpointTs(p.changefeed.Status) 443 ddlPuller := puller.NewPuller( 444 ctx, 445 ctx.GlobalVars().PDClient, 446 ctx.GlobalVars().GrpcPool, 447 ctx.GlobalVars().KVStorage, 448 checkpointTs, ddlspans, false) 449 meta, err := kv.GetSnapshotMeta(kvStorage, checkpointTs) 450 if err != nil { 451 return nil, errors.Trace(err) 452 } 453 schemaStorage, err := entry.NewSchemaStorage(meta, checkpointTs, p.filter, p.changefeed.Info.Config.ForceReplicate) 454 if err != nil { 455 return nil, errors.Trace(err) 456 } 457 p.wg.Add(1) 458 go func() { 459 defer p.wg.Done() 460 p.sendError(ddlPuller.Run(ctx)) 461 }() 462 ddlRawKVCh := puller.SortOutput(ctx, ddlPuller.Output()) 463 p.wg.Add(1) 464 go func() { 465 defer p.wg.Done() 466 var ddlRawKV *model.RawKVEntry 467 for { 468 select { 469 case <-ctx.Done(): 470 return 471 case ddlRawKV = <-ddlRawKVCh: 472 } 473 if ddlRawKV == nil { 474 continue 475 } 476 failpoint.Inject("processorDDLResolved", nil) 477 if ddlRawKV.OpType == model.OpTypeResolved { 478 schemaStorage.AdvanceResolvedTs(ddlRawKV.CRTs) 479 } 480 job, err := entry.UnmarshalDDL(ddlRawKV) 481 if err != nil { 482 p.sendError(errors.Trace(err)) 483 return 484 } 485 if job == nil { 486 continue 487 } 488 if err := schemaStorage.HandleDDLJob(job); err != nil { 489 p.sendError(errors.Trace(err)) 490 return 491 } 492 } 493 }() 494 return schemaStorage, nil 495 } 496 497 func (p *processor) sendError(err error) { 498 if err == nil { 499 return 500 } 501 select { 502 case p.errCh <- err: 503 default: 504 if errors.Cause(err) != context.Canceled { 505 log.Error("processor receives redundant error", zap.Error(err)) 506 } 507 } 508 } 509 510 // checkTablesNum if the number of table pipelines is equal to the number of TaskStatus in etcd state. 511 // if the table number is not right, create or remove the odd tables. 512 func (p *processor) checkTablesNum(ctx cdcContext.Context) error { 513 taskStatus := p.changefeed.TaskStatuses[p.captureInfo.ID] 514 if len(p.tables) == len(taskStatus.Tables) { 515 return nil 516 } 517 // check if a table should be listen but not 518 // this only could be happened in the first tick. 519 for tableID, replicaInfo := range taskStatus.Tables { 520 if _, exist := p.tables[tableID]; exist { 521 continue 522 } 523 opt := taskStatus.Operation 524 // TODO(leoppro): check if the operation is a undone add operation 525 if opt != nil && opt[tableID] != nil { 526 continue 527 } 528 log.Info("start to listen to the table immediately", zap.Int64("tableID", tableID), zap.Any("replicaInfo", replicaInfo)) 529 if replicaInfo.StartTs < p.changefeed.Status.CheckpointTs { 530 replicaInfo.StartTs = p.changefeed.Status.CheckpointTs 531 } 532 err := p.addTable(ctx, tableID, replicaInfo) 533 if err != nil { 534 return errors.Trace(err) 535 } 536 } 537 // check if a table should be removed but still exist 538 // this shouldn't be happened in any time. 539 for tableID, tablePipeline := range p.tables { 540 if _, exist := taskStatus.Tables[tableID]; exist { 541 continue 542 } 543 opt := taskStatus.Operation 544 if opt != nil && opt[tableID] != nil && opt[tableID].Delete { 545 // table will be removed by normal logic 546 continue 547 } 548 tablePipeline.Cancel() 549 tablePipeline.Wait() 550 delete(p.tables, tableID) 551 log.Warn("the table was forcibly deleted", zap.Int64("tableID", tableID), zap.Any("taskStatus", taskStatus)) 552 } 553 return nil 554 } 555 556 // handlePosition calculates the local resolved ts and local checkpoint ts 557 func (p *processor) handlePosition() { 558 minResolvedTs := uint64(math.MaxUint64) 559 if p.schemaStorage != nil { 560 minResolvedTs = p.schemaStorage.ResolvedTs() 561 } 562 for _, table := range p.tables { 563 ts := table.ResolvedTs() 564 if ts < minResolvedTs { 565 minResolvedTs = ts 566 } 567 } 568 569 minCheckpointTs := minResolvedTs 570 for _, table := range p.tables { 571 ts := table.CheckpointTs() 572 if ts < minCheckpointTs { 573 minCheckpointTs = ts 574 } 575 } 576 577 resolvedPhyTs := oracle.ExtractPhysical(minResolvedTs) 578 // It is more accurate to get tso from PD, but in most cases we have 579 // deployed NTP service, a little bias is acceptable here. 580 p.metricResolvedTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-resolvedPhyTs) / 1e3) 581 p.metricResolvedTsGauge.Set(float64(resolvedPhyTs)) 582 583 checkpointPhyTs := oracle.ExtractPhysical(minCheckpointTs) 584 // It is more accurate to get tso from PD, but in most cases we have 585 // deployed NTP service, a little bias is acceptable here. 586 p.metricCheckpointTsLagGauge.Set(float64(oracle.GetPhysical(time.Now())-checkpointPhyTs) / 1e3) 587 p.metricCheckpointTsGauge.Set(float64(checkpointPhyTs)) 588 589 // minResolvedTs and minCheckpointTs may less than global resolved ts and global checkpoint ts when a new table added, the startTs of the new table is less than global checkpoint ts. 590 if minResolvedTs != p.changefeed.TaskPositions[p.captureInfo.ID].ResolvedTs || 591 minCheckpointTs != p.changefeed.TaskPositions[p.captureInfo.ID].CheckPointTs { 592 p.changefeed.PatchTaskPosition(p.captureInfo.ID, func(position *model.TaskPosition) (*model.TaskPosition, bool, error) { 593 failpoint.Inject("ProcessorUpdatePositionDelaying", nil) 594 if position == nil { 595 // when the captureInfo is deleted, the old owner will delete task status, task position, task workload in non-atomic 596 // so processor may see a intermediate state, for example the task status is exist but task position is deleted. 597 log.Warn("task position is not exist, skip to update position", zap.String("changefeed", p.changefeed.ID)) 598 return nil, false, nil 599 } 600 position.CheckPointTs = minCheckpointTs 601 position.ResolvedTs = minResolvedTs 602 return position, true, nil 603 }) 604 } 605 } 606 607 // handleWorkload calculates the workload of all tables 608 func (p *processor) handleWorkload() { 609 p.changefeed.PatchTaskWorkload(p.captureInfo.ID, func(workloads model.TaskWorkload) (model.TaskWorkload, bool, error) { 610 changed := false 611 if workloads == nil { 612 workloads = make(model.TaskWorkload) 613 } 614 for tableID := range workloads { 615 if _, exist := p.tables[tableID]; !exist { 616 delete(workloads, tableID) 617 changed = true 618 } 619 } 620 for tableID, table := range p.tables { 621 if workloads[tableID] != table.Workload() { 622 workloads[tableID] = table.Workload() 623 changed = true 624 } 625 } 626 return workloads, changed, nil 627 }) 628 } 629 630 // pushResolvedTs2Table sends global resolved ts to all the table pipelines. 631 func (p *processor) pushResolvedTs2Table() { 632 resolvedTs := p.changefeed.Status.ResolvedTs 633 for _, table := range p.tables { 634 table.UpdateBarrierTs(resolvedTs) 635 } 636 } 637 638 // addTable creates a new table pipeline and adds it to the `p.tables` 639 func (p *processor) addTable(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) error { 640 if table, ok := p.tables[tableID]; ok { 641 if table.Status() == tablepipeline.TableStatusStopped { 642 log.Warn("The same table exists but is stopped. Cancel it and continue.", cdcContext.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) 643 table.Cancel() 644 table.Wait() 645 delete(p.tables, tableID) 646 } else { 647 log.Warn("Ignore existing table", cdcContext.ZapFieldChangefeed(ctx), zap.Int64("ID", tableID)) 648 return nil 649 } 650 } 651 652 globalCheckpointTs := p.changefeed.Status.CheckpointTs 653 654 if replicaInfo.StartTs < globalCheckpointTs { 655 log.Warn("addTable: startTs < checkpoint", 656 cdcContext.ZapFieldChangefeed(ctx), 657 zap.Int64("tableID", tableID), 658 zap.Uint64("checkpoint", globalCheckpointTs), 659 zap.Uint64("startTs", replicaInfo.StartTs)) 660 } 661 table, err := p.createTablePipeline(ctx, tableID, replicaInfo) 662 if err != nil { 663 return errors.Trace(err) 664 } 665 p.tables[tableID] = table 666 return nil 667 } 668 669 func (p *processor) createTablePipelineImpl(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) { 670 ctx = cdcContext.WithErrorHandler(ctx, func(err error) error { 671 if cerror.ErrTableProcessorStoppedSafely.Equal(err) || 672 errors.Cause(errors.Cause(err)) == context.Canceled { 673 return nil 674 } 675 p.sendError(err) 676 return nil 677 }) 678 var tableName *model.TableName 679 retry.Do(ctx, func() error { //nolint:errcheck 680 if name, ok := p.schemaStorage.GetLastSnapshot().GetTableNameByID(tableID); ok { 681 tableName = &name 682 return nil 683 } 684 return errors.Errorf("failed to get table name, fallback to use table id: %d", tableID) 685 }, retry.WithBackoffBaseDelay(backoffBaseDelayInMs), retry.WithMaxTries(maxTries), retry.WithIsRetryableErr(cerror.IsRetryableError)) 686 if p.changefeed.Info.Config.Cyclic.IsEnabled() { 687 // Retry to find mark table ID 688 var markTableID model.TableID 689 err := retry.Do(context.Background(), func() error { 690 if tableName == nil { 691 name, exist := p.schemaStorage.GetLastSnapshot().GetTableNameByID(tableID) 692 if !exist { 693 return cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%s)", tableID) 694 } 695 tableName = &name 696 } 697 markTableSchemaName, markTableTableName := mark.GetMarkTableName(tableName.Schema, tableName.Table) 698 tableInfo, exist := p.schemaStorage.GetLastSnapshot().GetTableByName(markTableSchemaName, markTableTableName) 699 if !exist { 700 return cerror.ErrProcessorTableNotFound.GenWithStack("normal table(%s) and mark table not match", tableName.String()) 701 } 702 markTableID = tableInfo.ID 703 return nil 704 }, retry.WithBackoffMaxDelay(50), retry.WithBackoffMaxDelay(60*1000), retry.WithMaxTries(20)) 705 if err != nil { 706 return nil, errors.Trace(err) 707 } 708 replicaInfo.MarkTableID = markTableID 709 } 710 var tableNameStr string 711 if tableName == nil { 712 log.Warn("failed to get table name for metric") 713 tableNameStr = strconv.Itoa(int(tableID)) 714 } else { 715 tableNameStr = tableName.QuoteString() 716 } 717 718 sink := p.sinkManager.CreateTableSink(tableID, replicaInfo.StartTs) 719 table := tablepipeline.NewTablePipeline( 720 ctx, 721 p.mounter, 722 tableID, 723 tableNameStr, 724 replicaInfo, 725 sink, 726 p.changefeed.Info.GetTargetTs(), 727 ) 728 p.wg.Add(1) 729 p.metricSyncTableNumGauge.Inc() 730 go func() { 731 table.Wait() 732 p.wg.Done() 733 p.metricSyncTableNumGauge.Dec() 734 log.Debug("Table pipeline exited", zap.Int64("tableID", tableID), 735 cdcContext.ZapFieldChangefeed(ctx), 736 zap.String("name", table.Name()), 737 zap.Any("replicaInfo", replicaInfo)) 738 }() 739 740 log.Info("Add table pipeline", zap.Int64("tableID", tableID), 741 cdcContext.ZapFieldChangefeed(ctx), 742 zap.String("name", table.Name()), 743 zap.Any("replicaInfo", replicaInfo), 744 zap.Uint64("globalResolvedTs", p.changefeed.Status.ResolvedTs)) 745 746 return table, nil 747 } 748 749 // doGCSchemaStorage trigger the schema storage GC 750 func (p *processor) doGCSchemaStorage() { 751 if p.schemaStorage == nil { 752 // schemaStorage is nil only in test 753 return 754 } 755 // Delay GC to accommodate pullers starting from a startTs that's too small 756 // TODO fix startTs problem and remove GC delay, or use other mechanism that prevents the problem deterministically 757 gcTime := oracle.GetTimeFromTS(p.changefeed.Status.CheckpointTs).Add(-schemaStorageGCLag) 758 gcTs := oracle.ComposeTS(gcTime.Unix(), 0) 759 p.schemaStorage.DoGC(gcTs) 760 } 761 762 func (p *processor) Close() error { 763 for _, tbl := range p.tables { 764 tbl.Cancel() 765 } 766 for _, tbl := range p.tables { 767 tbl.Wait() 768 } 769 p.cancel() 770 p.wg.Wait() 771 // mark tables share the same cdcContext with its original table, don't need to cancel 772 failpoint.Inject("processorStopDelay", nil) 773 resolvedTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) 774 resolvedTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) 775 checkpointTsGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) 776 checkpointTsLagGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) 777 syncTableNumGauge.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) 778 processorErrorCounter.DeleteLabelValues(p.changefeedID, p.captureInfo.AdvertiseAddr) 779 if p.sinkManager != nil { 780 // pass a canceled context is ok here, since we don't need to wait Close 781 ctx, cancel := context.WithCancel(context.Background()) 782 cancel() 783 return p.sinkManager.Close(ctx) 784 } 785 return nil 786 } 787 788 // WriteDebugInfo write the debug info to Writer 789 func (p *processor) WriteDebugInfo(w io.Writer) { 790 fmt.Fprintf(w, "%+v\n", *p.changefeed) 791 for tableID, tablePipeline := range p.tables { 792 fmt.Fprintf(w, "tableID: %d, tableName: %s, resolvedTs: %d, checkpointTs: %d, status: %s\n", 793 tableID, tablePipeline.Name(), tablePipeline.ResolvedTs(), tablePipeline.CheckpointTs(), tablePipeline.Status()) 794 } 795 }