github.com/pingcap/ticdc@v0.0.0-20220526033649-485a10ef2652/cdc/changefeed.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  	"math"
    20  	"sync"
    21  	"time"
    22  
    23  	"github.com/pingcap/errors"
    24  	"github.com/pingcap/failpoint"
    25  	"github.com/pingcap/log"
    26  	timodel "github.com/pingcap/parser/model"
    27  	"github.com/pingcap/ticdc/cdc/entry"
    28  	"github.com/pingcap/ticdc/cdc/kv"
    29  	"github.com/pingcap/ticdc/cdc/model"
    30  	"github.com/pingcap/ticdc/cdc/sink"
    31  	cdcContext "github.com/pingcap/ticdc/pkg/context"
    32  	"github.com/pingcap/ticdc/pkg/cyclic/mark"
    33  	cerror "github.com/pingcap/ticdc/pkg/errors"
    34  	"github.com/pingcap/ticdc/pkg/filter"
    35  	"github.com/pingcap/ticdc/pkg/scheduler"
    36  	"github.com/pingcap/tidb/sessionctx/binloginfo"
    37  	"go.etcd.io/etcd/clientv3"
    38  	"go.etcd.io/etcd/mvcc/mvccpb"
    39  	"go.uber.org/zap"
    40  )
    41  
    42  type tableIDMap = map[model.TableID]struct{}
    43  
    44  // OwnerDDLHandler defines the ddl handler for Owner
    45  // which can pull ddl jobs and execute ddl jobs
    46  type OwnerDDLHandler interface {
    47  	// PullDDL pulls the ddl jobs and returns resolvedTs of DDL Puller and job list.
    48  	PullDDL() (resolvedTs uint64, jobs []*timodel.Job, err error)
    49  
    50  	// Close cancels the executing of OwnerDDLHandler and releases resource
    51  	Close() error
    52  }
    53  
    54  // ChangeFeedRWriter defines the Reader and Writer for changeFeed
    55  type ChangeFeedRWriter interface {
    56  
    57  	// GetChangeFeeds returns kv revision and a map mapping from changefeedID to changefeed detail mvccpb.KeyValue
    58  	GetChangeFeeds(ctx context.Context) (int64, map[string]*mvccpb.KeyValue, error)
    59  
    60  	// GetAllTaskStatus queries all task status of a changefeed, and returns a map
    61  	// mapping from captureID to TaskStatus
    62  	GetAllTaskStatus(ctx context.Context, changefeedID string) (model.ProcessorsInfos, error)
    63  
    64  	// RemoveAllTaskStatus removes all task status of a changefeed
    65  	RemoveAllTaskStatus(ctx context.Context, changefeedID string) error
    66  
    67  	// GetAllTaskPositions queries all task positions of a changefeed, and returns a map
    68  	// mapping from captureID to TaskPositions
    69  	GetAllTaskPositions(ctx context.Context, changefeedID string) (map[string]*model.TaskPosition, error)
    70  
    71  	// RemoveAllTaskPositions removes all task partitions of a changefeed
    72  	RemoveAllTaskPositions(ctx context.Context, changefeedID string) error
    73  
    74  	// GetChangeFeedStatus queries the checkpointTs and resovledTs of a given changefeed
    75  	GetChangeFeedStatus(ctx context.Context, id string) (*model.ChangeFeedStatus, int64, error)
    76  
    77  	// PutAllChangeFeedStatus the changefeed info to storage such as etcd.
    78  	PutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus) error
    79  
    80  	// LeaseGuardRemoveAllTaskStatus wraps RemoveAllTaskStatus with a context restricted by lease TTL.
    81  	LeaseGuardRemoveAllTaskStatus(ctx context.Context, changefeedID string, leaseID clientv3.LeaseID) error
    82  
    83  	// LeaseGuardRemoveAllTaskPositions wraps RemoveAllTaskPositions with a context restricted by lease TTL.
    84  	LeaseGuardRemoveAllTaskPositions(ctx context.Context, changefeedID string, leaseID clientv3.LeaseID) error
    85  
    86  	// LeaseGuardPutAllChangeFeedStatus wraps PutAllChangeFeedStatus with a context restricted by lease TTL.
    87  	LeaseGuardPutAllChangeFeedStatus(ctx context.Context, infos map[model.ChangeFeedID]*model.ChangeFeedStatus, leaseID clientv3.LeaseID) error
    88  }
    89  
    90  type changeFeed struct {
    91  	id     string
    92  	info   *model.ChangeFeedInfo
    93  	status *model.ChangeFeedStatus
    94  	// The latest checkpointTs already applied to Etcd.
    95  	// We need to check this field to ensure visibility to the processors,
    96  	// if the operation assumes the progress of the global checkpoint.
    97  	appliedCheckpointTs uint64
    98  
    99  	schema           *entry.SingleSchemaSnapshot
   100  	ddlState         model.ChangeFeedDDLState
   101  	targetTs         uint64
   102  	ddlTs            uint64
   103  	syncpointMutex   sync.Mutex
   104  	updateResolvedTs bool
   105  	startTimer       chan bool
   106  	syncpointStore   sink.SyncpointStore
   107  	syncCancel       context.CancelFunc
   108  	taskStatus       model.ProcessorsInfos
   109  	taskPositions    map[model.CaptureID]*model.TaskPosition
   110  	filter           *filter.Filter
   111  	sink             AsyncSink
   112  	scheduler        scheduler.Scheduler
   113  
   114  	cyclicEnabled bool
   115  
   116  	ddlHandler    OwnerDDLHandler
   117  	ddlResolvedTs uint64
   118  	ddlJobHistory []*timodel.Job
   119  	ddlExecutedTs uint64
   120  	// ddlEventCache is not nil when the changefeed is executing a DDL event asynchronously
   121  	// After the DDL event has been executed, ddlEventCache will be set to nil.
   122  	ddlEventCache *model.DDLEvent
   123  
   124  	schemas map[model.SchemaID]tableIDMap
   125  	tables  map[model.TableID]model.TableName
   126  	// value of partitions is the slice of partitions ID.
   127  	partitions         map[model.TableID][]int64
   128  	orphanTables       map[model.TableID]model.Ts
   129  	toCleanTables      map[model.TableID]model.Ts
   130  	moveTableJobs      map[model.TableID]*model.MoveTableJob
   131  	manualMoveCommands []*model.MoveTableJob
   132  	rebalanceNextTick  bool
   133  
   134  	lastRebalanceTime time.Time
   135  
   136  	etcdCli kv.CDCEtcdClient
   137  	leaseID clientv3.LeaseID
   138  
   139  	// cdcCtx for all
   140  	cdcCtx cdcContext.Context
   141  	// context cancel function for all internal goroutines
   142  	cancel context.CancelFunc
   143  }
   144  
   145  // String implements fmt.Stringer interface.
   146  func (c *changeFeed) String() string {
   147  	format := "{\n ID: %s\n info: %+v\n status: %+v\n State: %v\n ProcessorInfos: %+v\n tables: %+v\n orphanTables: %+v\n toCleanTables: %v\n ddlResolvedTs: %d\n ddlJobHistory: %+v\n}\n\n"
   148  	s := fmt.Sprintf(format,
   149  		c.id, c.info, c.status, c.ddlState, c.taskStatus, c.tables,
   150  		c.orphanTables, c.toCleanTables, c.ddlResolvedTs, c.ddlJobHistory)
   151  
   152  	if len(c.ddlJobHistory) > 0 {
   153  		job := c.ddlJobHistory[0]
   154  		s += fmt.Sprintf("next to exec job: %s query: %s\n\n", job, job.Query)
   155  	}
   156  
   157  	return s
   158  }
   159  
   160  func (c *changeFeed) updateProcessorInfos(processInfos model.ProcessorsInfos, positions map[string]*model.TaskPosition) {
   161  	c.taskStatus = processInfos
   162  	c.taskPositions = positions
   163  }
   164  
   165  func (c *changeFeed) addSchema(schemaID model.SchemaID) {
   166  	if _, ok := c.schemas[schemaID]; ok {
   167  		log.Warn("add schema already exists", zap.Int64("schemaID", schemaID))
   168  		return
   169  	}
   170  	c.schemas[schemaID] = make(map[model.TableID]struct{})
   171  }
   172  
   173  func (c *changeFeed) dropSchema(schemaID model.SchemaID, targetTs model.Ts) {
   174  	if schema, ok := c.schemas[schemaID]; ok {
   175  		for tid := range schema {
   176  			c.removeTable(schemaID, tid, targetTs)
   177  		}
   178  	}
   179  	delete(c.schemas, schemaID)
   180  }
   181  
   182  func (c *changeFeed) addTable(tblInfo *model.TableInfo, targetTs model.Ts) {
   183  	if c.filter.ShouldIgnoreTable(tblInfo.TableName.Schema, tblInfo.TableName.Table) {
   184  		return
   185  	}
   186  	if c.cyclicEnabled && mark.IsMarkTable(tblInfo.TableName.Schema, tblInfo.TableName.Table) {
   187  		return
   188  	}
   189  
   190  	if _, ok := c.tables[tblInfo.ID]; ok {
   191  		log.Warn("add table already exists", zap.Int64("tableID", tblInfo.ID), zap.Stringer("table", tblInfo.TableName))
   192  		return
   193  	}
   194  
   195  	if !tblInfo.IsEligible(c.info.Config.ForceReplicate) {
   196  		log.Warn("skip ineligible table", zap.Int64("tid", tblInfo.ID), zap.Stringer("table", tblInfo.TableName))
   197  		return
   198  	}
   199  
   200  	if _, ok := c.schemas[tblInfo.SchemaID]; !ok {
   201  		c.schemas[tblInfo.SchemaID] = make(tableIDMap)
   202  	}
   203  	c.schemas[tblInfo.SchemaID][tblInfo.ID] = struct{}{}
   204  	c.tables[tblInfo.ID] = tblInfo.TableName
   205  	if pi := tblInfo.GetPartitionInfo(); pi != nil {
   206  		delete(c.partitions, tblInfo.ID)
   207  		for _, partition := range pi.Definitions {
   208  			c.partitions[tblInfo.ID] = append(c.partitions[tblInfo.ID], partition.ID)
   209  			c.orphanTables[partition.ID] = targetTs
   210  		}
   211  	} else {
   212  		c.orphanTables[tblInfo.ID] = targetTs
   213  	}
   214  }
   215  
   216  func (c *changeFeed) removeTable(sid model.SchemaID, tid model.TableID, targetTs model.Ts) {
   217  	if _, ok := c.schemas[sid]; ok {
   218  		delete(c.schemas[sid], tid)
   219  	}
   220  	delete(c.tables, tid)
   221  
   222  	removeFunc := func(id int64) {
   223  		if _, ok := c.orphanTables[id]; ok {
   224  			delete(c.orphanTables, id)
   225  		} else {
   226  			c.toCleanTables[id] = targetTs
   227  		}
   228  	}
   229  
   230  	if pids, ok := c.partitions[tid]; ok {
   231  		for _, id := range pids {
   232  			removeFunc(id)
   233  		}
   234  		delete(c.partitions, tid)
   235  	} else {
   236  		removeFunc(tid)
   237  	}
   238  }
   239  
   240  func (c *changeFeed) updatePartition(tblInfo *timodel.TableInfo, startTs uint64) {
   241  	tid := tblInfo.ID
   242  	partitionsID, ok := c.partitions[tid]
   243  	if !ok || len(partitionsID) == 0 {
   244  		return
   245  	}
   246  	oldIDs := make(map[int64]struct{}, len(partitionsID))
   247  	for _, pid := range partitionsID {
   248  		oldIDs[pid] = struct{}{}
   249  	}
   250  
   251  	pi := tblInfo.GetPartitionInfo()
   252  	if pi == nil {
   253  		return
   254  	}
   255  	newPartitionIDs := make([]int64, 0, len(pi.Definitions))
   256  	for _, partition := range pi.Definitions {
   257  		pid := partition.ID
   258  		_, ok := oldIDs[pid]
   259  		if !ok {
   260  			// new partition.
   261  			c.orphanTables[pid] = startTs
   262  		}
   263  		delete(oldIDs, partition.ID)
   264  		newPartitionIDs = append(newPartitionIDs, partition.ID)
   265  	}
   266  	// update the table partition IDs.
   267  	c.partitions[tid] = newPartitionIDs
   268  
   269  	// drop partition.
   270  	for pid := range oldIDs {
   271  		if _, ok := c.orphanTables[pid]; ok {
   272  			delete(c.orphanTables, pid)
   273  		} else {
   274  			c.toCleanTables[pid] = startTs
   275  		}
   276  	}
   277  }
   278  
   279  func (c *changeFeed) tryBalance(ctx context.Context, captures map[string]*model.CaptureInfo, rebalanceNow bool,
   280  	manualMoveCommands []*model.MoveTableJob) error {
   281  	err := c.balanceOrphanTables(ctx, captures)
   282  	if err != nil {
   283  		return errors.Trace(err)
   284  	}
   285  	c.manualMoveCommands = append(c.manualMoveCommands, manualMoveCommands...)
   286  	if rebalanceNow {
   287  		c.rebalanceNextTick = true
   288  	}
   289  	c.handleManualMoveTableJobs(captures)
   290  	err = c.rebalanceTables(ctx, captures)
   291  	if err != nil {
   292  		return errors.Trace(err)
   293  	}
   294  	err = c.handleMoveTableJobs(ctx, captures)
   295  	return errors.Trace(err)
   296  }
   297  
   298  func findTaskStatusWithTable(infos model.ProcessorsInfos, tableID model.TableID) (captureID model.CaptureID, info *model.TaskStatus, ok bool) {
   299  	for cid, info := range infos {
   300  		for tid := range info.Tables {
   301  			if tid == tableID {
   302  				return cid, info, true
   303  			}
   304  		}
   305  	}
   306  	return "", nil, false
   307  }
   308  
   309  func (c *changeFeed) balanceOrphanTables(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error {
   310  	if len(captures) == 0 {
   311  		return nil
   312  	}
   313  
   314  	// Do NOT rebalance orphan tables before checkpoint ts has advanced to FinishTs of a DDL
   315  	if c.appliedCheckpointTs != c.status.CheckpointTs {
   316  		return nil
   317  	}
   318  
   319  	captureIDs := make(map[model.CaptureID]struct{}, len(captures))
   320  	cleanedTables := make(map[model.TableID]struct{})
   321  	addedTables := make(map[model.TableID]struct{})
   322  	updateFuncs := make(map[model.CaptureID][]kv.UpdateTaskStatusFunc)
   323  	for cid := range captures {
   324  		captureIDs[cid] = struct{}{}
   325  	}
   326  	c.scheduler.AlignCapture(captureIDs)
   327  
   328  	for id, targetTs := range c.toCleanTables {
   329  		captureID, _, ok := findTaskStatusWithTable(c.taskStatus, id)
   330  		if !ok {
   331  			log.Warn("ignore clean table id", zap.Int64("id", id))
   332  			delete(c.toCleanTables, id)
   333  			continue
   334  		}
   335  
   336  		id := id
   337  		targetTs := targetTs
   338  		updateFuncs[captureID] = append(updateFuncs[captureID], func(_ int64, status *model.TaskStatus) (bool, error) {
   339  			status.RemoveTable(id, targetTs, false /*isMoveTable*/)
   340  			return true, nil
   341  		})
   342  		cleanedTables[id] = struct{}{}
   343  	}
   344  
   345  	operations := c.scheduler.DistributeTables(c.orphanTables)
   346  	for captureID, operation := range operations {
   347  		schemaSnapshot := c.schema
   348  		for tableID, op := range operation {
   349  			var orphanMarkTableID model.TableID
   350  			tableName, found := schemaSnapshot.GetTableNameByID(tableID)
   351  			if !found {
   352  				log.Warn("balance orphan tables delay, table not found",
   353  					zap.String("changefeed", c.id),
   354  					zap.Int64("tableID", tableID))
   355  				continue
   356  			}
   357  			if c.cyclicEnabled {
   358  				markTableSchameName, markTableTableName := mark.GetMarkTableName(tableName.Schema, tableName.Table)
   359  				orphanMarkTableID, found = schemaSnapshot.GetTableIDByName(markTableSchameName, markTableTableName)
   360  				if !found {
   361  					// Mark table is not created yet, skip and wait.
   362  					log.Info("balance orphan tables delay, wait mark table",
   363  						zap.String("changefeed", c.id),
   364  						zap.Int64("tableID", tableID),
   365  						zap.String("markTableName", markTableTableName))
   366  					continue
   367  				}
   368  			}
   369  			info := &model.TableReplicaInfo{
   370  				StartTs:     op.BoundaryTs,
   371  				MarkTableID: orphanMarkTableID,
   372  			}
   373  			tableID := tableID
   374  			op := op
   375  			updateFuncs[captureID] = append(updateFuncs[captureID], func(_ int64, status *model.TaskStatus) (bool, error) {
   376  				status.AddTable(tableID, info, op.BoundaryTs)
   377  				return true, nil
   378  			})
   379  			addedTables[tableID] = struct{}{}
   380  		}
   381  	}
   382  
   383  	for captureID, funcs := range updateFuncs {
   384  		newStatus, _, err := c.etcdCli.LeaseGuardAtomicPutTaskStatus(ctx, c.id, captureID, c.leaseID, funcs...)
   385  		if err != nil {
   386  			return errors.Trace(err)
   387  		}
   388  		c.taskStatus[captureID] = newStatus.Clone()
   389  		log.Info("dispatch table success", zap.String("capture-id", captureID), zap.Stringer("status", newStatus))
   390  		failpoint.Inject("OwnerRemoveTableError", func() {
   391  			if len(cleanedTables) > 0 {
   392  				failpoint.Return(errors.New("failpoint injected error"))
   393  			}
   394  		})
   395  	}
   396  
   397  	for tableID := range cleanedTables {
   398  		delete(c.toCleanTables, tableID)
   399  	}
   400  	for tableID := range addedTables {
   401  		delete(c.orphanTables, tableID)
   402  	}
   403  
   404  	return nil
   405  }
   406  
   407  func (c *changeFeed) updateTaskStatus(ctx context.Context, taskStatus map[model.CaptureID]*model.TaskStatus) error {
   408  	for captureID, status := range taskStatus {
   409  		newStatus, _, err := c.etcdCli.LeaseGuardAtomicPutTaskStatus(
   410  			ctx, c.id, captureID, c.leaseID,
   411  			func(modRevision int64, taskStatus *model.TaskStatus) (bool, error) {
   412  				if taskStatus.SomeOperationsUnapplied() {
   413  					log.Error("unexpected task status, there are operations unapplied in this status", zap.Any("status", taskStatus))
   414  					return false, cerror.ErrWaitHandleOperationTimeout.GenWithStackByArgs()
   415  				}
   416  				taskStatus.Tables = status.Tables
   417  				taskStatus.Operation = status.Operation
   418  				return true, nil
   419  			})
   420  		if err != nil {
   421  			return errors.Trace(err)
   422  		}
   423  		c.taskStatus[captureID] = newStatus.Clone()
   424  		log.Info("dispatch table success", zap.String("capture-id", captureID), zap.Stringer("status", status))
   425  	}
   426  	return nil
   427  }
   428  
   429  func (c *changeFeed) handleManualMoveTableJobs(captures map[model.CaptureID]*model.CaptureInfo) {
   430  	if len(captures) == 0 {
   431  		return
   432  	}
   433  	if len(c.moveTableJobs) > 0 {
   434  		return
   435  	}
   436  	for len(c.manualMoveCommands) > 0 {
   437  		moveJob := c.manualMoveCommands[0]
   438  		if _, exist := c.moveTableJobs[moveJob.TableID]; exist {
   439  			break
   440  		}
   441  		c.manualMoveCommands = c.manualMoveCommands[1:]
   442  		moveJob.From = ""
   443  		for captureID, taskStatus := range c.taskStatus {
   444  			if _, exist := taskStatus.Tables[moveJob.TableID]; exist {
   445  				moveJob.From = captureID
   446  				break
   447  			}
   448  		}
   449  		if moveJob.From == "" {
   450  			log.Warn("invalid manual move job, the table is not found", zap.Reflect("job", moveJob))
   451  			continue
   452  		}
   453  		if moveJob.To == moveJob.From {
   454  			log.Warn("invalid manual move job, the table is already exists in the target capture", zap.Reflect("job", moveJob))
   455  			continue
   456  		}
   457  		if _, exist := captures[moveJob.To]; !exist {
   458  			log.Warn("invalid manual move job, the target capture is not found", zap.Reflect("job", moveJob))
   459  			continue
   460  		}
   461  		if c.moveTableJobs == nil {
   462  			c.moveTableJobs = make(map[model.TableID]*model.MoveTableJob)
   463  		}
   464  		c.moveTableJobs[moveJob.TableID] = moveJob
   465  		log.Info("received the manual move table job", zap.Reflect("job", moveJob))
   466  	}
   467  }
   468  
   469  func (c *changeFeed) rebalanceTables(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error {
   470  	if len(captures) == 0 {
   471  		return nil
   472  	}
   473  	if len(c.moveTableJobs) != 0 {
   474  		return nil
   475  	}
   476  	for _, status := range c.taskStatus {
   477  		if status.SomeOperationsUnapplied() {
   478  			return nil
   479  		}
   480  	}
   481  	timeToRebalance := time.Since(c.lastRebalanceTime) > time.Duration(c.info.Config.Scheduler.PollingTime)*time.Minute
   482  	timeToRebalance = timeToRebalance && c.info.Config.Scheduler.PollingTime > 0
   483  
   484  	if !c.rebalanceNextTick && !timeToRebalance {
   485  		return nil
   486  	}
   487  	c.lastRebalanceTime = time.Now()
   488  	c.rebalanceNextTick = false
   489  
   490  	captureIDs := make(map[model.CaptureID]struct{}, len(captures))
   491  	for cid := range captures {
   492  		captureIDs[cid] = struct{}{}
   493  		workloads, err := c.etcdCli.GetTaskWorkload(ctx, c.id, cid)
   494  		if err != nil {
   495  			return errors.Trace(err)
   496  		}
   497  		c.scheduler.ResetWorkloads(cid, workloads)
   498  	}
   499  	c.scheduler.AlignCapture(captureIDs)
   500  
   501  	_, moveTableJobs := c.scheduler.CalRebalanceOperates(0)
   502  	log.Info("rebalance operations", zap.Reflect("moveTableJobs", moveTableJobs))
   503  	c.moveTableJobs = moveTableJobs
   504  	return nil
   505  }
   506  
   507  func (c *changeFeed) handleMoveTableJobs(ctx context.Context, captures map[model.CaptureID]*model.CaptureInfo) error {
   508  	if len(captures) == 0 {
   509  		return nil
   510  	}
   511  	if len(c.moveTableJobs) == 0 {
   512  		return nil
   513  	}
   514  	for _, status := range c.taskStatus {
   515  		if status.SomeOperationsUnapplied() {
   516  			return nil
   517  		}
   518  	}
   519  	newTaskStatus := make(map[model.CaptureID]*model.TaskStatus, len(captures))
   520  	cloneStatus := func(captureID model.CaptureID) (*model.TaskStatus, bool) {
   521  		status, exist := newTaskStatus[captureID]
   522  		if !exist {
   523  			taskStatus := c.taskStatus[captureID]
   524  			if taskStatus == nil {
   525  				if _, exist := captures[captureID]; !exist {
   526  					return nil, false
   527  				}
   528  				status = new(model.TaskStatus)
   529  			} else {
   530  				status = taskStatus.Clone()
   531  			}
   532  			newTaskStatus[captureID] = status
   533  		}
   534  		return status, true
   535  	}
   536  	for tableID, job := range c.moveTableJobs {
   537  		switch job.Status {
   538  		case model.MoveTableStatusNone:
   539  			// delete table from original capture
   540  			status, exist := cloneStatus(job.From)
   541  			if !exist {
   542  				delete(c.moveTableJobs, tableID)
   543  				log.Warn("ignored the move job, the source capture is not found", zap.Reflect("job", job))
   544  				continue
   545  			}
   546  			// To ensure that the replication pipeline stops exactly at the boundary TS,
   547  			// The boundary TS specified by Remove Table Operation MUST greater or equal to the checkpoint TS of this table.
   548  			// So the global resolved TS is a reasonable values.
   549  			replicaInfo, exist := status.RemoveTable(tableID, c.status.ResolvedTs, true /*isMoveTable*/)
   550  			if !exist {
   551  				delete(c.moveTableJobs, tableID)
   552  				log.Warn("ignored the move job, the table is not exist in the source capture", zap.Reflect("job", job))
   553  				continue
   554  			}
   555  			replicaInfo.StartTs = c.status.ResolvedTs
   556  			job.TableReplicaInfo = replicaInfo
   557  			job.Status = model.MoveTableStatusDeleted
   558  			log.Info("handle the move job, remove table from the source capture", zap.Reflect("job", job))
   559  		case model.MoveTableStatusDeleted:
   560  			// Do NOT dispatch tables before checkpoint ts has been flushed to Etcd.
   561  			if c.appliedCheckpointTs != c.status.CheckpointTs {
   562  				log.Debug("handle the move job, waiting for checkpoint ts to be uploaded",
   563  					zap.Uint64("applied-checkpoint-ts", c.appliedCheckpointTs),
   564  					zap.Uint64("latest-checkpoint-ts", c.status.CheckpointTs))
   565  				continue
   566  			}
   567  
   568  			// add table to target capture
   569  			status, exist := cloneStatus(job.To)
   570  			replicaInfo := job.TableReplicaInfo.Clone()
   571  			if replicaInfo.StartTs < c.status.CheckpointTs {
   572  				replicaInfo.StartTs = c.status.CheckpointTs
   573  			}
   574  			if !exist {
   575  				// the target capture is not exist, add table to orphanTables.
   576  				c.orphanTables[tableID] = replicaInfo.StartTs
   577  				delete(c.moveTableJobs, tableID)
   578  				log.Warn("the target capture is not exist, sent the table to orphanTables", zap.Reflect("job", job))
   579  				continue
   580  			}
   581  			status.AddTable(tableID, replicaInfo, replicaInfo.StartTs)
   582  			job.Status = model.MoveTableStatusFinished
   583  			delete(c.moveTableJobs, tableID)
   584  			log.Info("handle the move job, add table to the target capture", zap.Reflect("job", job))
   585  		}
   586  	}
   587  	err := c.updateTaskStatus(ctx, newTaskStatus)
   588  	return errors.Trace(err)
   589  }
   590  
   591  func (c *changeFeed) applyJob(job *timodel.Job) (skip bool, err error) {
   592  	schemaID := job.SchemaID
   593  	if job.BinlogInfo != nil && job.BinlogInfo.TableInfo != nil && c.schema.IsIneligibleTableID(job.BinlogInfo.TableInfo.ID) {
   594  		tableID := job.BinlogInfo.TableInfo.ID
   595  		if _, exist := c.tables[tableID]; exist {
   596  			c.removeTable(schemaID, tableID, job.BinlogInfo.FinishedTS)
   597  		}
   598  		return true, nil
   599  	}
   600  
   601  	err = func() error {
   602  		// case table id set may change
   603  		switch job.Type {
   604  		case timodel.ActionCreateSchema:
   605  			c.addSchema(schemaID)
   606  		case timodel.ActionDropSchema:
   607  			c.dropSchema(schemaID, job.BinlogInfo.FinishedTS)
   608  		case timodel.ActionCreateTable, timodel.ActionRecoverTable:
   609  			addID := job.BinlogInfo.TableInfo.ID
   610  			table, exist := c.schema.TableByID(addID)
   611  			if !exist {
   612  				return cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(addID)
   613  			}
   614  			c.addTable(table, job.BinlogInfo.FinishedTS)
   615  		case timodel.ActionDropTable:
   616  			dropID := job.TableID
   617  			c.removeTable(schemaID, dropID, job.BinlogInfo.FinishedTS)
   618  		case timodel.ActionRenameTable:
   619  			tableName, exist := c.schema.GetTableNameByID(job.TableID)
   620  			if !exist {
   621  				return cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(job.TableID)
   622  			}
   623  			// no id change just update name
   624  			c.tables[job.TableID] = tableName
   625  		case timodel.ActionTruncateTable:
   626  			dropID := job.TableID
   627  			c.removeTable(schemaID, dropID, job.BinlogInfo.FinishedTS)
   628  
   629  			addID := job.BinlogInfo.TableInfo.ID
   630  			table, exist := c.schema.TableByID(addID)
   631  			if !exist {
   632  				return cerror.ErrSnapshotTableNotFound.GenWithStackByArgs(addID)
   633  			}
   634  			c.addTable(table, job.BinlogInfo.FinishedTS)
   635  		case timodel.ActionTruncateTablePartition, timodel.ActionAddTablePartition, timodel.ActionDropTablePartition:
   636  			c.updatePartition(job.BinlogInfo.TableInfo, job.BinlogInfo.FinishedTS)
   637  		}
   638  		return nil
   639  	}()
   640  	if err != nil {
   641  		log.Error("failed to applyJob, start to print debug info", zap.Error(err))
   642  		c.schema.PrintStatus(log.Error)
   643  	}
   644  	return false, err
   645  }
   646  
   647  // handleDDL check if we can change the status to be `ChangeFeedExecDDL` and execute the DDL asynchronously
   648  // if the status is in ChangeFeedWaitToExecDDL.
   649  // After executing the DDL successfully, the status will be changed to be ChangeFeedSyncDML.
   650  func (c *changeFeed) handleDDL() error {
   651  	// async ddl
   652  	if c.ddlState == model.ChangeFeedExecDDL && c.ddlEventCache != nil {
   653  		done, err := c.sink.EmitDDLEvent(c.cdcCtx, c.ddlEventCache)
   654  		if err != nil {
   655  			return cerror.ErrExecDDLFailed.GenWithStackByArgs()
   656  		}
   657  		if done {
   658  			c.ddlExecutedTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS
   659  			c.ddlJobHistory = c.ddlJobHistory[1:]
   660  			c.ddlState = model.ChangeFeedSyncDML
   661  			c.ddlEventCache = nil
   662  		}
   663  		return nil
   664  	}
   665  
   666  	if c.ddlState != model.ChangeFeedWaitToExecDDL {
   667  		return nil
   668  	}
   669  
   670  	if len(c.ddlJobHistory) == 0 {
   671  		log.Panic("ddl job history can not be empty in changefeed when should to execute DDL")
   672  	}
   673  	todoDDLJob := c.ddlJobHistory[0]
   674  
   675  	// Check if all the checkpointTs of capture are achieving global resolvedTs(which is equal to todoDDLJob.FinishedTS)
   676  	if len(c.taskStatus) > len(c.taskPositions) {
   677  		return nil
   678  	}
   679  
   680  	if c.appliedCheckpointTs < todoDDLJob.BinlogInfo.FinishedTS-1 {
   681  		log.Debug("wait checkpoint ts",
   682  			zap.Uint64("checkpoint ts", c.status.CheckpointTs),
   683  			zap.Uint64("applied checkpoint ts", c.appliedCheckpointTs),
   684  			zap.Uint64("finish ts", todoDDLJob.BinlogInfo.FinishedTS),
   685  			zap.String("ddl query", todoDDLJob.Query))
   686  		return nil
   687  	}
   688  
   689  	if c.appliedCheckpointTs >= todoDDLJob.BinlogInfo.FinishedTS {
   690  		log.Panic("applied checkpoint ts is larger than DDL finish ts",
   691  			zap.Uint64("applied checkpoint ts", c.appliedCheckpointTs),
   692  			zap.Uint64("finish ts", todoDDLJob.BinlogInfo.FinishedTS))
   693  	}
   694  
   695  	log.Info("apply job", zap.Stringer("job", todoDDLJob),
   696  		zap.String("schema", todoDDLJob.SchemaName),
   697  		zap.String("query", todoDDLJob.Query),
   698  		zap.Uint64("start-ts", todoDDLJob.StartTS),
   699  		zap.Uint64("ts", todoDDLJob.BinlogInfo.FinishedTS))
   700  
   701  	ddlEvent := new(model.DDLEvent)
   702  	preTableInfo, err := c.schema.PreTableInfo(todoDDLJob)
   703  	if err != nil {
   704  		return errors.Trace(err)
   705  	}
   706  	err = c.schema.HandleDDL(todoDDLJob)
   707  	if err != nil {
   708  		return errors.Trace(err)
   709  	}
   710  	err = c.schema.FillSchemaName(todoDDLJob)
   711  	if err != nil {
   712  		return errors.Trace(err)
   713  	}
   714  
   715  	ddlEvent.FromJob(todoDDLJob, preTableInfo)
   716  
   717  	// Execute DDL Job asynchronously
   718  	c.ddlState = model.ChangeFeedExecDDL
   719  
   720  	// TODO consider some newly added DDL types such as `ActionCreateSequence`
   721  	skip, err := c.applyJob(todoDDLJob)
   722  	if err != nil {
   723  		return errors.Trace(err)
   724  	}
   725  	if skip {
   726  		log.Info("ddl job ignored", zap.String("changefeed", c.id), zap.Reflect("job", todoDDLJob))
   727  		c.ddlJobHistory = c.ddlJobHistory[1:]
   728  		c.ddlExecutedTs = todoDDLJob.BinlogInfo.FinishedTS
   729  		c.ddlState = model.ChangeFeedSyncDML
   730  		c.ddlEventCache = nil
   731  		return nil
   732  	}
   733  
   734  	if !c.cyclicEnabled || c.info.Config.Cyclic.SyncDDL {
   735  		failpoint.Inject("InjectChangefeedDDLError", func() {
   736  			failpoint.Return(cerror.ErrExecDDLFailed.GenWithStackByArgs())
   737  		})
   738  
   739  		ddlEvent.Query = binloginfo.AddSpecialComment(ddlEvent.Query)
   740  		log.Debug("DDL processed to make special features mysql-compatible", zap.String("query", ddlEvent.Query))
   741  		c.ddlEventCache = ddlEvent
   742  		done, err := c.sink.EmitDDLEvent(c.cdcCtx, c.ddlEventCache)
   743  		// If DDL executing failed, pause the changefeed and print log, rather
   744  		// than return an error and break the running of this owner.
   745  		if err != nil {
   746  			return cerror.ErrExecDDLFailed.GenWithStackByArgs()
   747  		}
   748  		if done {
   749  			c.ddlJobHistory = c.ddlJobHistory[1:]
   750  			c.ddlExecutedTs = todoDDLJob.BinlogInfo.FinishedTS
   751  			c.ddlState = model.ChangeFeedSyncDML
   752  			c.ddlEventCache = nil
   753  		}
   754  		return nil
   755  
   756  	}
   757  	log.Info("Execute DDL ignored", zap.String("changefeed", c.id), zap.Reflect("ddlJob", todoDDLJob))
   758  	c.ddlJobHistory = c.ddlJobHistory[1:]
   759  	c.ddlExecutedTs = todoDDLJob.BinlogInfo.FinishedTS
   760  	c.ddlState = model.ChangeFeedSyncDML
   761  	return nil
   762  }
   763  
   764  // handleSyncPoint record every syncpoint to downstream if the syncpoint feature is enable
   765  func (c *changeFeed) handleSyncPoint(ctx context.Context) error {
   766  	// sync-point on
   767  	if c.info.SyncPointEnabled {
   768  		c.syncpointMutex.Lock()
   769  		defer c.syncpointMutex.Unlock()
   770  		// ticker and ddl can trigger syncpoint record at the same time, only record once
   771  		syncpointRecorded := false
   772  		// ResolvedTs == CheckpointTs means a syncpoint reached;
   773  		// !c.updateResolvedTs means the syncpoint is setted by ticker;
   774  		// c.ddlTs == 0 means no DDL wait to exec and we can sink the syncpoint record securely ( c.ddlTs != 0 means some DDL should be sink to downstream and this syncpoint is fake ).
   775  		if c.status.ResolvedTs == c.status.CheckpointTs && !c.updateResolvedTs {
   776  			log.Info("sync point reached by ticker", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs))
   777  			c.updateResolvedTs = true
   778  			err := c.syncpointStore.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs)
   779  			if err != nil {
   780  				log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err))
   781  				return err
   782  			}
   783  			syncpointRecorded = true
   784  		}
   785  
   786  		if c.status.ResolvedTs == 0 {
   787  			c.updateResolvedTs = true
   788  		}
   789  
   790  		// ResolvedTs == CheckpointTs means a syncpoint reached;
   791  		// ResolvedTs == ddlTs means the syncpoint is setted by DDL;
   792  		// ddlTs <= ddlExecutedTs means the DDL has been execed.
   793  		if c.status.ResolvedTs == c.status.CheckpointTs && c.status.ResolvedTs == c.ddlTs && c.ddlTs <= c.ddlExecutedTs {
   794  			log.Info("sync point reached by ddl", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Bool("updateResolvedTs", c.updateResolvedTs), zap.Uint64("ddlResolvedTs", c.ddlResolvedTs), zap.Uint64("ddlTs", c.ddlTs), zap.Uint64("ddlExecutedTs", c.ddlExecutedTs))
   795  			if !syncpointRecorded {
   796  				err := c.syncpointStore.SinkSyncpoint(ctx, c.id, c.status.CheckpointTs)
   797  				if err != nil {
   798  					log.Error("syncpoint sink fail", zap.Uint64("ResolvedTs", c.status.ResolvedTs), zap.Uint64("CheckpointTs", c.status.CheckpointTs), zap.Error(err))
   799  					return err
   800  				}
   801  			}
   802  			c.ddlTs = 0
   803  		}
   804  	}
   805  	return nil
   806  }
   807  
   808  // calcResolvedTs update every changefeed's resolve ts and checkpoint ts.
   809  func (c *changeFeed) calcResolvedTs() error {
   810  	if c.ddlState != model.ChangeFeedSyncDML && c.ddlState != model.ChangeFeedWaitToExecDDL {
   811  		log.Debug("skip update resolved ts", zap.String("ddlState", c.ddlState.String()))
   812  		return nil
   813  	}
   814  
   815  	minResolvedTs := c.targetTs
   816  	minCheckpointTs := c.targetTs
   817  
   818  	// prevMinResolvedTs and prevMinCheckpointTs are used for debug
   819  	prevMinResolvedTs := c.targetTs
   820  	prevMinCheckpointTs := c.targetTs
   821  	checkUpdateTs := func() {
   822  		if prevMinCheckpointTs != minCheckpointTs {
   823  			log.L().WithOptions(zap.AddCallerSkip(1)).Debug("min checkpoint updated",
   824  				zap.Uint64("prevMinCheckpointTs", prevMinCheckpointTs),
   825  				zap.Uint64("minCheckpointTs", minCheckpointTs))
   826  			prevMinCheckpointTs = minCheckpointTs
   827  		}
   828  		if prevMinResolvedTs != minResolvedTs {
   829  			log.L().WithOptions(zap.AddCallerSkip(1)).Debug("min resolved updated",
   830  				zap.Uint64("prevMinResolvedTs", prevMinResolvedTs),
   831  				zap.Uint64("minResolvedTs", minResolvedTs))
   832  			prevMinResolvedTs = minResolvedTs
   833  		}
   834  	}
   835  
   836  	if len(c.taskPositions) < len(c.taskStatus) {
   837  		log.Debug("skip update resolved ts",
   838  			zap.Int("taskPositions", len(c.taskPositions)),
   839  			zap.Int("taskStatus", len(c.taskStatus)))
   840  		return nil
   841  	}
   842  	if len(c.taskPositions) == 0 {
   843  		minCheckpointTs = c.status.ResolvedTs
   844  	} else {
   845  		// calc the min of all resolvedTs in captures
   846  		for _, position := range c.taskPositions {
   847  			if minResolvedTs > position.ResolvedTs {
   848  				minResolvedTs = position.ResolvedTs
   849  			}
   850  
   851  			if minCheckpointTs > position.CheckPointTs {
   852  				minCheckpointTs = position.CheckPointTs
   853  			}
   854  		}
   855  	}
   856  	prevMinCheckpointTs = minCheckpointTs
   857  	prevMinResolvedTs = minResolvedTs
   858  
   859  	for captureID, status := range c.taskStatus {
   860  		appliedTs := status.AppliedTs()
   861  		if minCheckpointTs > appliedTs {
   862  			minCheckpointTs = appliedTs
   863  		}
   864  		if minResolvedTs > appliedTs {
   865  			minResolvedTs = appliedTs
   866  		}
   867  		if appliedTs != math.MaxUint64 {
   868  			log.Debug("some operation is still unapplied",
   869  				zap.String("capture-id", captureID),
   870  				zap.Uint64("appliedTs", appliedTs),
   871  				zap.Stringer("status", status))
   872  		}
   873  	}
   874  	checkUpdateTs()
   875  
   876  	for _, startTs := range c.orphanTables {
   877  		if minCheckpointTs > startTs {
   878  			minCheckpointTs = startTs
   879  		}
   880  		if minResolvedTs > startTs {
   881  			minResolvedTs = startTs
   882  		}
   883  	}
   884  	checkUpdateTs()
   885  
   886  	for _, targetTs := range c.toCleanTables {
   887  		if minCheckpointTs > targetTs {
   888  			minCheckpointTs = targetTs
   889  		}
   890  		if minResolvedTs > targetTs {
   891  			minResolvedTs = targetTs
   892  		}
   893  	}
   894  	checkUpdateTs()
   895  
   896  	// if minResolvedTs is greater than ddlResolvedTs,
   897  	// it means that ddlJobHistory in memory is not intact,
   898  	// there are some ddl jobs which finishedTs is smaller than minResolvedTs we don't know.
   899  	// so we need to call `pullDDLJob`, update the ddlJobHistory and ddlResolvedTs.
   900  	if minResolvedTs > c.ddlResolvedTs {
   901  		if err := c.pullDDLJob(); err != nil {
   902  			return errors.Trace(err)
   903  		}
   904  
   905  		if minResolvedTs > c.ddlResolvedTs {
   906  			minResolvedTs = c.ddlResolvedTs
   907  		}
   908  	}
   909  	checkUpdateTs()
   910  
   911  	// if minResolvedTs is greater than the finishedTS of ddl job which is not executed,
   912  	// we need to execute this ddl job
   913  	for len(c.ddlJobHistory) > 0 && c.ddlJobHistory[0].BinlogInfo.FinishedTS <= c.ddlExecutedTs {
   914  		c.ddlJobHistory = c.ddlJobHistory[1:]
   915  	}
   916  	if len(c.ddlJobHistory) > 0 && minResolvedTs >= c.ddlJobHistory[0].BinlogInfo.FinishedTS {
   917  		minResolvedTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS
   918  		c.ddlState = model.ChangeFeedWaitToExecDDL
   919  		c.ddlTs = minResolvedTs
   920  	}
   921  
   922  	if len(c.ddlJobHistory) > 0 && minCheckpointTs >= c.ddlJobHistory[0].BinlogInfo.FinishedTS {
   923  		minCheckpointTs = c.ddlJobHistory[0].BinlogInfo.FinishedTS - 1
   924  	}
   925  
   926  	// if downstream sink is the MQ sink, the MQ sink do not promise that checkpoint is less than globalResolvedTs
   927  	if minCheckpointTs > minResolvedTs {
   928  		minCheckpointTs = minResolvedTs
   929  	}
   930  	checkUpdateTs()
   931  
   932  	var tsUpdated bool
   933  
   934  	// syncpoint on
   935  	if c.info.SyncPointEnabled {
   936  		c.syncpointMutex.Lock()
   937  		if c.updateResolvedTs && minResolvedTs > c.status.ResolvedTs {
   938  			c.status.ResolvedTs = minResolvedTs
   939  			tsUpdated = true
   940  		}
   941  		c.syncpointMutex.Unlock()
   942  	} else if minResolvedTs > c.status.ResolvedTs {
   943  		c.status.ResolvedTs = minResolvedTs
   944  		tsUpdated = true
   945  	}
   946  
   947  	if minCheckpointTs > c.status.CheckpointTs {
   948  		c.status.CheckpointTs = minCheckpointTs
   949  		// when the `c.ddlState` is `model.ChangeFeedWaitToExecDDL`,
   950  		// some DDL is waiting to executed, we can't ensure whether the DDL has been executed.
   951  		// so we can't emit checkpoint to sink
   952  		if c.ddlState != model.ChangeFeedWaitToExecDDL {
   953  			failpoint.Inject("InjectEmitCheckpointTsError", func() {
   954  				failpoint.Return(cerror.ErrEmitCheckpointTsFailed.GenWithStackByArgs())
   955  			})
   956  			err := c.sink.EmitCheckpointTs(c.cdcCtx, minCheckpointTs)
   957  			if err != nil {
   958  				return err
   959  			}
   960  		}
   961  		tsUpdated = true
   962  	}
   963  	checkUpdateTs()
   964  
   965  	if tsUpdated {
   966  		log.Debug("update changefeed", zap.String("id", c.id),
   967  			zap.Uint64("checkpoint ts", c.status.CheckpointTs),
   968  			zap.Uint64("resolved ts", c.status.ResolvedTs))
   969  	}
   970  	return nil
   971  }
   972  
   973  func (c *changeFeed) pullDDLJob() error {
   974  	ddlResolvedTs, ddlJobs, err := c.ddlHandler.PullDDL()
   975  	if err != nil {
   976  		return errors.Trace(err)
   977  	}
   978  	c.ddlResolvedTs = ddlResolvedTs
   979  	for _, ddl := range ddlJobs {
   980  		if c.filter.ShouldDiscardDDL(ddl.Type) {
   981  			log.Info("discard the ddl job", zap.Int64("jobID", ddl.ID), zap.String("query", ddl.Query))
   982  			continue
   983  		}
   984  		c.ddlJobHistory = append(c.ddlJobHistory, ddl)
   985  	}
   986  	return nil
   987  }
   988  
   989  // startSyncPeriod start a timer for every changefeed to create sync point by time
   990  func (c *changeFeed) startSyncPeriod(ctx context.Context, interval time.Duration) {
   991  	log.Debug("sync ticker start", zap.Duration("sync-interval", interval))
   992  	go func(ctx context.Context) {
   993  		ticker := time.NewTicker(interval)
   994  		for {
   995  			select {
   996  			case <-ctx.Done():
   997  				return
   998  			case <-ticker.C:
   999  				c.syncpointMutex.Lock()
  1000  				c.updateResolvedTs = false
  1001  				c.syncpointMutex.Unlock()
  1002  			}
  1003  		}
  1004  	}(ctx)
  1005  }
  1006  
  1007  func (c *changeFeed) stopSyncPointTicker() {
  1008  	if c.syncCancel != nil {
  1009  		c.syncCancel()
  1010  		c.syncCancel = nil
  1011  	}
  1012  }
  1013  
  1014  func (c *changeFeed) startSyncPointTicker(ctx context.Context, interval time.Duration) {
  1015  	var syncCtx context.Context
  1016  	syncCtx, c.syncCancel = context.WithCancel(ctx)
  1017  	c.startSyncPeriod(syncCtx, interval)
  1018  }
  1019  
  1020  func (c *changeFeed) Close() {
  1021  	if c.ddlHandler != nil {
  1022  		err := c.ddlHandler.Close()
  1023  		if err != nil && errors.Cause(err) != context.Canceled {
  1024  			log.Warn("failed to close ddl handler", zap.Error(err))
  1025  		}
  1026  	}
  1027  
  1028  	ctx, cancel := context.WithCancel(context.Background())
  1029  	cancel()
  1030  	if c.sink != nil {
  1031  		// pass a canceled context is enough, since the Close of backend sink
  1032  		// here doesn't use context actually.
  1033  		err := c.sink.Close(ctx)
  1034  		if err != nil && errors.Cause(err) != context.Canceled {
  1035  			log.Warn("failed to close owner sink", zap.Error(err))
  1036  		}
  1037  	}
  1038  
  1039  	if c.syncpointStore != nil {
  1040  		err := c.syncpointStore.Close()
  1041  		if err != nil && errors.Cause(err) != context.Canceled {
  1042  			log.Warn("failed to close owner sink", zap.Error(err))
  1043  		}
  1044  	}
  1045  
  1046  	if c.cancel != nil {
  1047  		c.cancel()
  1048  	}
  1049  	log.Info("changefeed closed", zap.String("id", c.id))
  1050  }