github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/sinkmanager/manager.go (about)

     1  // Copyright 2022 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 sinkmanager
    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  	"github.com/pingcap/tiflow/cdc/entry"
    27  	"github.com/pingcap/tiflow/cdc/model"
    28  	"github.com/pingcap/tiflow/cdc/processor/memquota"
    29  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager"
    30  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager/sorter"
    31  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    32  	"github.com/pingcap/tiflow/cdc/redo"
    33  	"github.com/pingcap/tiflow/cdc/sink/dmlsink/factory"
    34  	tablesinkmetrics "github.com/pingcap/tiflow/cdc/sink/metrics/tablesink"
    35  	"github.com/pingcap/tiflow/cdc/sink/tablesink"
    36  	"github.com/pingcap/tiflow/pkg/config"
    37  	pconfig "github.com/pingcap/tiflow/pkg/config"
    38  	cerror "github.com/pingcap/tiflow/pkg/errors"
    39  	"github.com/pingcap/tiflow/pkg/retry"
    40  	"github.com/pingcap/tiflow/pkg/spanz"
    41  	"github.com/pingcap/tiflow/pkg/upstream"
    42  	"github.com/pingcap/tiflow/pkg/util"
    43  	"github.com/prometheus/client_golang/prometheus"
    44  	"github.com/tikv/client-go/v2/oracle"
    45  	"go.uber.org/zap"
    46  	"golang.org/x/sync/errgroup"
    47  )
    48  
    49  const (
    50  	sinkWorkerNum               = 8
    51  	redoWorkerNum               = 4
    52  	defaultGenerateTaskInterval = 100 * time.Millisecond
    53  	// sorter.CleanByTable can be expensive. So it's necessary to reduce useless calls.
    54  	cleanTableInterval  = 5 * time.Second
    55  	cleanTableMinEvents = 128
    56  )
    57  
    58  // TableStats of a table sink.
    59  type TableStats struct {
    60  	CheckpointTs model.Ts
    61  	ResolvedTs   model.Ts
    62  	LastSyncedTs model.Ts
    63  	BarrierTs    model.Ts
    64  }
    65  
    66  // SinkManager is the implementation of SinkManager.
    67  type SinkManager struct {
    68  	changefeedID model.ChangeFeedID
    69  
    70  	sinkURI string
    71  	config  *pconfig.ReplicaConfig
    72  
    73  	// up is the upstream and used to get the current pd time.
    74  	up *upstream.Upstream
    75  
    76  	// used to generate task upperbounds.
    77  	schemaStorage entry.SchemaStorage
    78  
    79  	// sinkProgressHeap is the heap of the table progress for sink.
    80  	sinkProgressHeap *tableProgresses
    81  	// redoProgressHeap is the heap of the table progress for redo.
    82  	redoProgressHeap *tableProgresses
    83  
    84  	// redoDMLMgr is used to report the resolved ts of the table if redo log is enabled.
    85  	redoDMLMgr redo.DMLManager
    86  	// sourceManager is used by the sink manager to fetch data.
    87  	sourceManager *sourcemanager.SourceManager
    88  
    89  	// sinkFactory used to create table sink.
    90  	sinkFactory struct {
    91  		sync.Mutex
    92  		f *factory.SinkFactory
    93  		// When every time we want to create a new factory, version will be increased and
    94  		// errors will be replaced by a new channel. version is used to distinct different
    95  		// sink factories in table sinks.
    96  		version uint64
    97  		errors  chan error
    98  	}
    99  
   100  	// tableSinks is a map from tableID to tableSink.
   101  	tableSinks spanz.SyncMap
   102  
   103  	// sinkWorkers used to pull data from source manager.
   104  	sinkWorkers []*sinkWorker
   105  	// sinkTaskChan is used to send tasks to sinkWorkers.
   106  	sinkTaskChan        chan *sinkTask
   107  	sinkWorkerAvailable chan struct{}
   108  	// sinkMemQuota is used to control the total memory usage of the table sink.
   109  	sinkMemQuota *memquota.MemQuota
   110  	sinkRetry    *retry.ErrorRetry
   111  	// redoWorkers used to pull data from source manager.
   112  	redoWorkers []*redoWorker
   113  	// redoTaskChan is used to send tasks to redoWorkers.
   114  	redoTaskChan        chan *redoTask
   115  	redoWorkerAvailable chan struct{}
   116  	// redoMemQuota is used to control the total memory usage of the redo.
   117  	redoMemQuota *memquota.MemQuota
   118  
   119  	// To control lifetime of all sub-goroutines.
   120  	managerCtx    context.Context
   121  	managerCancel context.CancelFunc
   122  	ready         chan struct{}
   123  
   124  	// To control lifetime of sink and redo tasks.
   125  	sinkEg *errgroup.Group
   126  	redoEg *errgroup.Group
   127  
   128  	// wg is used to wait for all workers to exit.
   129  	wg sync.WaitGroup
   130  
   131  	// Metric for table sink.
   132  	metricsTableSinkTotalRows prometheus.Counter
   133  
   134  	metricsTableSinkFlushLagDuration prometheus.Observer
   135  }
   136  
   137  // New creates a new sink manager.
   138  func New(
   139  	changefeedID model.ChangeFeedID,
   140  	sinkURI string,
   141  	config *pconfig.ReplicaConfig,
   142  	up *upstream.Upstream,
   143  	schemaStorage entry.SchemaStorage,
   144  	redoDMLMgr redo.DMLManager,
   145  	sourceManager *sourcemanager.SourceManager,
   146  ) *SinkManager {
   147  	m := &SinkManager{
   148  		changefeedID:        changefeedID,
   149  		up:                  up,
   150  		schemaStorage:       schemaStorage,
   151  		sourceManager:       sourceManager,
   152  		sinkURI:             sinkURI,
   153  		config:              config,
   154  		sinkProgressHeap:    newTableProgresses(),
   155  		sinkWorkers:         make([]*sinkWorker, 0, sinkWorkerNum),
   156  		sinkTaskChan:        make(chan *sinkTask),
   157  		sinkWorkerAvailable: make(chan struct{}, 1),
   158  		sinkRetry:           retry.NewInfiniteErrorRetry(),
   159  
   160  		metricsTableSinkTotalRows: tablesinkmetrics.TotalRowsCountCounter.
   161  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   162  
   163  		metricsTableSinkFlushLagDuration: tablesinkmetrics.TableSinkFlushLagDuration.
   164  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   165  	}
   166  
   167  	totalQuota := config.MemoryQuota
   168  	if redoDMLMgr != nil && redoDMLMgr.Enabled() {
   169  		m.redoDMLMgr = redoDMLMgr
   170  		m.redoProgressHeap = newTableProgresses()
   171  		m.redoWorkers = make([]*redoWorker, 0, redoWorkerNum)
   172  		m.redoTaskChan = make(chan *redoTask)
   173  		m.redoWorkerAvailable = make(chan struct{}, 1)
   174  
   175  		consistentMemoryUsage := m.config.Consistent.MemoryUsage
   176  		if consistentMemoryUsage == nil {
   177  			consistentMemoryUsage = pconfig.GetDefaultReplicaConfig().Consistent.MemoryUsage
   178  		}
   179  
   180  		redoQuota := totalQuota * consistentMemoryUsage.MemoryQuotaPercentage / 100
   181  		sinkQuota := totalQuota - redoQuota
   182  		m.sinkMemQuota = memquota.NewMemQuota(changefeedID, sinkQuota, "sink")
   183  		m.redoMemQuota = memquota.NewMemQuota(changefeedID, redoQuota, "redo")
   184  	} else {
   185  		m.sinkMemQuota = memquota.NewMemQuota(changefeedID, totalQuota, "sink")
   186  		m.redoMemQuota = memquota.NewMemQuota(changefeedID, 0, "redo")
   187  	}
   188  
   189  	m.ready = make(chan struct{})
   190  	return m
   191  }
   192  
   193  // Run implements util.Runnable.
   194  // When it returns, all sub-goroutines should be closed.
   195  func (m *SinkManager) Run(ctx context.Context, warnings ...chan<- error) (err error) {
   196  	m.managerCtx, m.managerCancel = context.WithCancel(ctx)
   197  	m.wg.Add(1) // So `SinkManager.Close` will also wait the function.
   198  	defer func() {
   199  		m.wg.Done()
   200  		m.waitSubroutines()
   201  		log.Info("Sink manager exists",
   202  			zap.String("namespace", m.changefeedID.Namespace),
   203  			zap.String("changefeed", m.changefeedID.ID),
   204  			zap.Error(err))
   205  	}()
   206  
   207  	splitTxn := util.GetOrZero(m.config.Sink.TxnAtomicity).ShouldSplitTxn()
   208  
   209  	gcErrors := make(chan error, 16)
   210  	sinkErrors := make(chan error, 16)
   211  	redoErrors := make(chan error, 16)
   212  
   213  	m.backgroundGC(gcErrors)
   214  	if m.sinkEg == nil {
   215  		var sinkCtx context.Context
   216  		m.sinkEg, sinkCtx = errgroup.WithContext(m.managerCtx)
   217  		m.startSinkWorkers(sinkCtx, m.sinkEg, splitTxn)
   218  		m.sinkEg.Go(func() error { return m.generateSinkTasks(sinkCtx) })
   219  		m.wg.Add(1)
   220  		go func() {
   221  			defer m.wg.Done()
   222  			if err := m.sinkEg.Wait(); err != nil && !cerror.Is(err, context.Canceled) {
   223  				log.Error("Worker handles or generates sink task failed",
   224  					zap.String("namespace", m.changefeedID.Namespace),
   225  					zap.String("changefeed", m.changefeedID.ID),
   226  					zap.Error(err))
   227  				select {
   228  				case sinkErrors <- err:
   229  				case <-m.managerCtx.Done():
   230  				}
   231  			}
   232  		}()
   233  	}
   234  	if m.redoDMLMgr != nil && m.redoEg == nil {
   235  		var redoCtx context.Context
   236  		m.redoEg, redoCtx = errgroup.WithContext(m.managerCtx)
   237  		m.startRedoWorkers(redoCtx, m.redoEg)
   238  		m.redoEg.Go(func() error { return m.generateRedoTasks(redoCtx) })
   239  		m.wg.Add(1)
   240  		go func() {
   241  			defer m.wg.Done()
   242  			if err := m.redoEg.Wait(); err != nil && !cerror.Is(err, context.Canceled) {
   243  				log.Error("Worker handles or generates redo task failed",
   244  					zap.String("namespace", m.changefeedID.Namespace),
   245  					zap.String("changefeed", m.changefeedID.ID),
   246  					zap.Error(err))
   247  				select {
   248  				case redoErrors <- err:
   249  				case <-m.managerCtx.Done():
   250  				}
   251  			}
   252  		}()
   253  	}
   254  
   255  	close(m.ready)
   256  	log.Info("Sink manager is created",
   257  		zap.String("namespace", m.changefeedID.Namespace),
   258  		zap.String("changefeed", m.changefeedID.ID),
   259  		zap.Bool("withRedoEnabled", m.redoDMLMgr != nil))
   260  
   261  	// SinkManager will restart some internal modules if necessasry.
   262  	for {
   263  		sinkFactoryErrors, sinkFactoryVersion := m.initSinkFactory()
   264  
   265  		select {
   266  		case <-m.managerCtx.Done():
   267  			return m.managerCtx.Err()
   268  		case err = <-gcErrors:
   269  			return errors.Trace(err)
   270  		case err = <-sinkErrors:
   271  			return errors.Trace(err)
   272  		case err = <-redoErrors:
   273  			return errors.Trace(err)
   274  		case err = <-sinkFactoryErrors:
   275  			log.Warn("Sink manager backend sink fails",
   276  				zap.String("namespace", m.changefeedID.Namespace),
   277  				zap.String("changefeed", m.changefeedID.ID),
   278  				zap.Uint64("factoryVersion", sinkFactoryVersion),
   279  				zap.Error(err))
   280  			m.clearSinkFactory()
   281  
   282  			// To release memory quota ASAP, close all table sinks manually.
   283  			start := time.Now()
   284  			log.Info("Sink manager is closing all table sinks",
   285  				zap.String("namespace", m.changefeedID.Namespace),
   286  				zap.String("changefeed", m.changefeedID.ID))
   287  			m.tableSinks.Range(func(span tablepb.Span, value interface{}) bool {
   288  				value.(*tableSinkWrapper).closeTableSink()
   289  				m.sinkMemQuota.ClearTable(span)
   290  				return true
   291  			})
   292  			log.Info("Sink manager has closed all table sinks",
   293  				zap.String("namespace", m.changefeedID.Namespace),
   294  				zap.String("changefeed", m.changefeedID.ID),
   295  				zap.Duration("cost", time.Since(start)))
   296  
   297  			// For duplicate entry error, we fast fail to restart changefeed.
   298  			if cerror.IsDupEntryError(err) {
   299  				return errors.Trace(err)
   300  			}
   301  		}
   302  
   303  		// If the error is retryable, we should retry to re-establish the internal resources.
   304  		if !cerror.ShouldFailChangefeed(err) && errors.Cause(err) != context.Canceled {
   305  			select {
   306  			case <-m.managerCtx.Done():
   307  			case warnings[0] <- err:
   308  			}
   309  		} else {
   310  			return errors.Trace(err)
   311  		}
   312  
   313  		backoff, err := m.sinkRetry.GetRetryBackoff(err)
   314  		if err != nil {
   315  			return errors.New(fmt.Sprintf("GetRetryBackoff: %s", err.Error()))
   316  		}
   317  
   318  		if err = util.Hang(m.managerCtx, backoff); err != nil {
   319  			return errors.Trace(err)
   320  		}
   321  	}
   322  }
   323  
   324  func (m *SinkManager) needsStuckCheck() bool {
   325  	m.sinkFactory.Lock()
   326  	defer m.sinkFactory.Unlock()
   327  	return m.sinkFactory.f != nil && m.sinkFactory.f.Category() == factory.CategoryMQ
   328  }
   329  
   330  func (m *SinkManager) initSinkFactory() (chan error, uint64) {
   331  	m.sinkFactory.Lock()
   332  	defer m.sinkFactory.Unlock()
   333  	uri := m.sinkURI
   334  	cfg := m.config
   335  
   336  	if m.sinkFactory.f != nil {
   337  		return m.sinkFactory.errors, m.sinkFactory.version
   338  	}
   339  	if m.sinkFactory.errors == nil {
   340  		m.sinkFactory.errors = make(chan error, 16)
   341  		m.sinkFactory.version += 1
   342  	}
   343  
   344  	emitError := func(err error) {
   345  		select {
   346  		case <-m.managerCtx.Done():
   347  		case m.sinkFactory.errors <- err:
   348  		}
   349  	}
   350  
   351  	var err error = nil
   352  	failpoint.Inject("SinkManagerRunError", func() {
   353  		log.Info("failpoint SinkManagerRunError injected", zap.String("changefeed", m.changefeedID.ID))
   354  		err = errors.New("SinkManagerRunError")
   355  	})
   356  	if err != nil {
   357  		emitError(err)
   358  		return m.sinkFactory.errors, m.sinkFactory.version
   359  	}
   360  
   361  	m.sinkFactory.f, err = factory.New(m.managerCtx, m.changefeedID, uri, cfg, m.sinkFactory.errors, m.up.PDClock)
   362  	if err != nil {
   363  		emitError(err)
   364  		return m.sinkFactory.errors, m.sinkFactory.version
   365  	}
   366  
   367  	log.Info("Sink manager inits sink factory success",
   368  		zap.String("namespace", m.changefeedID.Namespace),
   369  		zap.String("changefeed", m.changefeedID.ID),
   370  		zap.Uint64("factoryVersion", m.sinkFactory.version))
   371  	return m.sinkFactory.errors, m.sinkFactory.version
   372  }
   373  
   374  func (m *SinkManager) clearSinkFactory() {
   375  	m.sinkFactory.Lock()
   376  	defer m.sinkFactory.Unlock()
   377  	if m.sinkFactory.f != nil {
   378  		log.Info("Sink manager closing sink factory",
   379  			zap.String("namespace", m.changefeedID.Namespace),
   380  			zap.String("changefeed", m.changefeedID.ID),
   381  			zap.Uint64("factoryVersion", m.sinkFactory.version))
   382  		m.sinkFactory.f.Close()
   383  		m.sinkFactory.f = nil
   384  		log.Info("Sink manager has closed sink factory",
   385  			zap.String("namespace", m.changefeedID.Namespace),
   386  			zap.String("changefeed", m.changefeedID.ID),
   387  			zap.Uint64("factoryVersion", m.sinkFactory.version))
   388  	}
   389  	if m.sinkFactory.errors != nil {
   390  		close(m.sinkFactory.errors)
   391  		for range m.sinkFactory.errors {
   392  		}
   393  		m.sinkFactory.errors = nil
   394  	}
   395  }
   396  
   397  func (m *SinkManager) putSinkFactoryError(err error, version uint64) (success bool) {
   398  	m.sinkFactory.Lock()
   399  	defer m.sinkFactory.Unlock()
   400  	if version == m.sinkFactory.version {
   401  		select {
   402  		case m.sinkFactory.errors <- err:
   403  		default:
   404  		}
   405  		return true
   406  	}
   407  	return false
   408  }
   409  
   410  func (m *SinkManager) startSinkWorkers(ctx context.Context, eg *errgroup.Group, splitTxn bool) {
   411  	for i := 0; i < sinkWorkerNum; i++ {
   412  		w := newSinkWorker(m.changefeedID, m.sourceManager,
   413  			m.sinkMemQuota, splitTxn)
   414  		m.sinkWorkers = append(m.sinkWorkers, w)
   415  		eg.Go(func() error { return w.handleTasks(ctx, m.sinkTaskChan) })
   416  	}
   417  }
   418  
   419  func (m *SinkManager) startRedoWorkers(ctx context.Context, eg *errgroup.Group) {
   420  	for i := 0; i < redoWorkerNum; i++ {
   421  		w := newRedoWorker(m.changefeedID, m.sourceManager, m.redoMemQuota,
   422  			m.redoDMLMgr)
   423  		m.redoWorkers = append(m.redoWorkers, w)
   424  		eg.Go(func() error { return w.handleTasks(ctx, m.redoTaskChan) })
   425  	}
   426  }
   427  
   428  // backgroundGC is used to clean up the old data in the sorter.
   429  func (m *SinkManager) backgroundGC(errors chan<- error) {
   430  	ticker := time.NewTicker(time.Second)
   431  	m.wg.Add(1)
   432  	go func() {
   433  		defer m.wg.Done()
   434  		defer ticker.Stop()
   435  		for {
   436  			select {
   437  			case <-m.managerCtx.Done():
   438  				log.Info("Background GC is stopped because context is canceled",
   439  					zap.String("namespace", m.changefeedID.Namespace),
   440  					zap.String("changefeed", m.changefeedID.ID))
   441  				return
   442  			case <-ticker.C:
   443  				tableSinks := spanz.NewHashMap[*tableSinkWrapper]()
   444  				m.tableSinks.Range(func(key tablepb.Span, value any) bool {
   445  					wrapper := value.(*tableSinkWrapper)
   446  					tableSinks.ReplaceOrInsert(key, wrapper)
   447  					return true
   448  				})
   449  
   450  				tableSinks.Range(func(span tablepb.Span, sink *tableSinkWrapper) bool {
   451  					if time.Since(sink.lastCleanTime) < cleanTableInterval {
   452  						return true
   453  					}
   454  					checkpointTs := sink.getCheckpointTs()
   455  					resolvedMark := checkpointTs.ResolvedMark()
   456  					if resolvedMark == 0 {
   457  						return true
   458  					}
   459  
   460  					cleanPos := sorter.Position{StartTs: resolvedMark - 1, CommitTs: resolvedMark}
   461  					if !sink.cleanRangeEventCounts(cleanPos, cleanTableMinEvents) {
   462  						return true
   463  					}
   464  
   465  					if err := m.sourceManager.CleanByTable(span, cleanPos); err != nil {
   466  						log.Error("Failed to clean table in sort engine",
   467  							zap.String("namespace", m.changefeedID.Namespace),
   468  							zap.String("changefeed", m.changefeedID.ID),
   469  							zap.Stringer("span", &span),
   470  							zap.Error(err))
   471  						select {
   472  						case errors <- err:
   473  						case <-m.managerCtx.Done():
   474  						}
   475  					} else {
   476  						log.Debug("table stale data has been cleaned",
   477  							zap.String("namespace", m.changefeedID.Namespace),
   478  							zap.String("changefeed", m.changefeedID.ID),
   479  							zap.Stringer("span", &span),
   480  							zap.Any("upperBound", cleanPos))
   481  					}
   482  					sink.lastCleanTime = time.Now()
   483  					return true
   484  				})
   485  			}
   486  		}
   487  	}()
   488  }
   489  
   490  func (m *SinkManager) getUpperBound(tableSinkUpperBoundTs model.Ts) sorter.Position {
   491  	schemaTs := m.schemaStorage.ResolvedTs()
   492  	if schemaTs != math.MaxUint64 && tableSinkUpperBoundTs > schemaTs+1 {
   493  		// schemaTs == math.MaxUint64 means it's in tests.
   494  		tableSinkUpperBoundTs = schemaTs + 1
   495  	}
   496  	return sorter.Position{StartTs: tableSinkUpperBoundTs - 1, CommitTs: tableSinkUpperBoundTs}
   497  }
   498  
   499  // generateSinkTasks generates tasks to fetch data from the source manager.
   500  func (m *SinkManager) generateSinkTasks(ctx context.Context) error {
   501  	dispatchTasks := func() error {
   502  		tables := make([]*tableSinkWrapper, 0, sinkWorkerNum)
   503  		progs := make([]*progress, 0, sinkWorkerNum)
   504  
   505  		// Collect some table progresses.
   506  		for len(tables) < sinkWorkerNum && m.sinkProgressHeap.len() > 0 {
   507  			slowestTableProgress := m.sinkProgressHeap.pop()
   508  			span := slowestTableProgress.span
   509  
   510  			value, ok := m.tableSinks.Load(span)
   511  			if !ok {
   512  				log.Info("Table sink not found, probably already removed",
   513  					zap.String("namespace", m.changefeedID.Namespace),
   514  					zap.String("changefeed", m.changefeedID.ID),
   515  					zap.Stringer("span", &span))
   516  				// Maybe the table sink is removed by the processor.(Scheduled the table to other nodes.)
   517  				// So we do **not** need add it back to the heap.
   518  				continue
   519  			}
   520  			tableSink := value.(*tableSinkWrapper)
   521  			if tableSink.version != slowestTableProgress.version {
   522  				// The progress maybe stale.
   523  				continue
   524  			}
   525  
   526  			tableState := tableSink.getState()
   527  			// It means table sink is stopping or stopped.
   528  			// We should skip it and do not push it back.
   529  			// Because there is no case that stopping/stopped -> replicating.
   530  			if tableState != tablepb.TableStateReplicating {
   531  				log.Info("Table sink is not replicating, skip it",
   532  					zap.String("namespace", m.changefeedID.Namespace),
   533  					zap.String("changefeed", m.changefeedID.ID),
   534  					zap.Stringer("span", &span),
   535  					zap.String("tableState", tableState.String()))
   536  				continue
   537  			}
   538  			tables = append(tables, tableSink)
   539  			progs = append(progs, slowestTableProgress)
   540  		}
   541  
   542  		i := 0
   543  	LOOP:
   544  		for ; i < len(tables); i++ {
   545  			tableSink := tables[i]
   546  			slowestTableProgress := progs[i]
   547  			lowerBound := slowestTableProgress.nextLowerBoundPos
   548  			upperBound := m.getUpperBound(tableSink.getUpperBoundTs())
   549  
   550  			if !tableSink.initTableSink() {
   551  				// The table hasn't been attached to a sink.
   552  				m.sinkProgressHeap.push(slowestTableProgress)
   553  				continue
   554  			}
   555  
   556  			if sinkErr := tableSink.checkTableSinkHealth(); sinkErr != nil {
   557  				switch errors.Cause(sinkErr).(type) {
   558  				case tablesink.SinkInternalError:
   559  					tableSink.closeAndClearTableSink()
   560  					if restartErr := tableSink.restart(ctx); restartErr == nil {
   561  						// Restart the table sink based on the checkpoint position.
   562  						ckpt := tableSink.getCheckpointTs().ResolvedMark()
   563  						lastWrittenPos := sorter.Position{StartTs: ckpt - 1, CommitTs: ckpt}
   564  						p := &progress{
   565  							span:              tableSink.span,
   566  							nextLowerBoundPos: lastWrittenPos.Next(),
   567  							version:           slowestTableProgress.version,
   568  						}
   569  						m.sinkProgressHeap.push(p)
   570  						log.Info("table sink has been restarted",
   571  							zap.String("namespace", m.changefeedID.Namespace),
   572  							zap.String("changefeed", m.changefeedID.ID),
   573  							zap.Stringer("span", &tableSink.span),
   574  							zap.Any("lastWrittenPos", lastWrittenPos),
   575  							zap.String("sinkError", sinkErr.Error()))
   576  					} else {
   577  						m.sinkProgressHeap.push(slowestTableProgress)
   578  						log.Warn("table sink restart fail",
   579  							zap.String("namespace", m.changefeedID.Namespace),
   580  							zap.String("changefeed", m.changefeedID.ID),
   581  							zap.Stringer("span", &tableSink.span),
   582  							zap.String("sinkError", sinkErr.Error()),
   583  							zap.Error(restartErr))
   584  					}
   585  				default:
   586  					return sinkErr
   587  				}
   588  				continue
   589  			}
   590  
   591  			// The table has no available progress.
   592  			if lowerBound.Compare(upperBound) >= 0 {
   593  				m.sinkProgressHeap.push(slowestTableProgress)
   594  				continue
   595  			}
   596  
   597  			// No available memory, skip this round directly.
   598  			if !m.sinkMemQuota.TryAcquire(requestMemSize) {
   599  				break LOOP
   600  			}
   601  
   602  			log.Debug("MemoryQuotaTracing: try acquire memory for table sink task",
   603  				zap.String("namespace", m.changefeedID.Namespace),
   604  				zap.String("changefeed", m.changefeedID.ID),
   605  				zap.Stringer("span", &tableSink.span),
   606  				zap.Uint64("memory", requestMemSize))
   607  
   608  			t := &sinkTask{
   609  				span:          tableSink.span,
   610  				lowerBound:    lowerBound,
   611  				getUpperBound: m.getUpperBound,
   612  				tableSink:     tableSink,
   613  				callback: func(lastWrittenPos sorter.Position) {
   614  					p := &progress{
   615  						span:              tableSink.span,
   616  						nextLowerBoundPos: lastWrittenPos.Next(),
   617  						version:           slowestTableProgress.version,
   618  					}
   619  					m.sinkProgressHeap.push(p)
   620  					select {
   621  					case m.sinkWorkerAvailable <- struct{}{}:
   622  					default:
   623  					}
   624  				},
   625  				isCanceled: func() bool {
   626  					return tableSink.getState() != tablepb.TableStateReplicating
   627  				},
   628  			}
   629  			select {
   630  			case <-ctx.Done():
   631  				return ctx.Err()
   632  			case m.sinkTaskChan <- t:
   633  				log.Debug("Generate sink task",
   634  					zap.String("namespace", m.changefeedID.Namespace),
   635  					zap.String("changefeed", m.changefeedID.ID),
   636  					zap.Stringer("span", &tableSink.span),
   637  					zap.Any("lowerBound", lowerBound),
   638  					zap.Any("currentUpperBound", upperBound))
   639  			default:
   640  				m.sinkMemQuota.Refund(requestMemSize)
   641  				log.Debug("MemoryQuotaTracing: refund memory for table sink task",
   642  					zap.String("namespace", m.changefeedID.Namespace),
   643  					zap.String("changefeed", m.changefeedID.ID),
   644  					zap.Stringer("span", &tableSink.span),
   645  					zap.Uint64("memory", requestMemSize))
   646  				break LOOP
   647  			}
   648  		}
   649  		// Some progresses are not handled, return them back.
   650  		for ; i < len(progs); i++ {
   651  			m.sinkProgressHeap.push(progs[i])
   652  		}
   653  		return nil
   654  	}
   655  
   656  	taskTicker := time.NewTicker(defaultGenerateTaskInterval)
   657  	defer taskTicker.Stop()
   658  	for {
   659  		select {
   660  		case <-ctx.Done():
   661  			return ctx.Err()
   662  		case <-taskTicker.C:
   663  			if err := dispatchTasks(); err != nil {
   664  				return errors.Trace(err)
   665  			}
   666  		case <-m.sinkWorkerAvailable:
   667  			if err := dispatchTasks(); err != nil {
   668  				return errors.Trace(err)
   669  			}
   670  		}
   671  	}
   672  }
   673  
   674  func (m *SinkManager) generateRedoTasks(ctx context.Context) error {
   675  	dispatchTasks := func() error {
   676  		tables := make([]*tableSinkWrapper, 0, redoWorkerNum)
   677  		progs := make([]*progress, 0, redoWorkerNum)
   678  
   679  		for len(tables) < redoWorkerNum && m.redoProgressHeap.len() > 0 {
   680  			slowestTableProgress := m.redoProgressHeap.pop()
   681  			span := slowestTableProgress.span
   682  
   683  			value, ok := m.tableSinks.Load(span)
   684  			if !ok {
   685  				log.Info("Table sink not found, probably already removed",
   686  					zap.String("namespace", m.changefeedID.Namespace),
   687  					zap.String("changefeed", m.changefeedID.ID),
   688  					zap.Stringer("span", &span))
   689  				// Maybe the table sink is removed by the processor.(Scheduled the table to other nodes.)
   690  				// So we do **not** need add it back to the heap.
   691  				continue
   692  			}
   693  			tableSink := value.(*tableSinkWrapper)
   694  			if tableSink.version != slowestTableProgress.version {
   695  				// The progress maybe stale.
   696  				continue
   697  			}
   698  
   699  			tableState := tableSink.getState()
   700  			// It means table sink is stopping or stopped.
   701  			// We should skip it and do not push it back.
   702  			// Because there is no case that stopping/stopped -> replicating.
   703  			if tableState != tablepb.TableStateReplicating {
   704  				log.Info("Table sink is not replicating, skip it",
   705  					zap.String("namespace", m.changefeedID.Namespace),
   706  					zap.String("changefeed", m.changefeedID.ID),
   707  					zap.Stringer("span", &span),
   708  					zap.String("tableState", tableState.String()))
   709  				continue
   710  			}
   711  			tables = append(tables, tableSink)
   712  			progs = append(progs, slowestTableProgress)
   713  		}
   714  
   715  		i := 0
   716  	LOOP:
   717  		for ; i < len(tables); i++ {
   718  			tableSink := tables[i]
   719  			slowestTableProgress := progs[i]
   720  			lowerBound := slowestTableProgress.nextLowerBoundPos
   721  			upperBound := m.getUpperBound(tableSink.getReceivedSorterResolvedTs())
   722  
   723  			// The table has no available progress.
   724  			if lowerBound.Compare(upperBound) >= 0 {
   725  				m.redoProgressHeap.push(slowestTableProgress)
   726  				continue
   727  			}
   728  
   729  			// No available memory, skip this round directly.
   730  			if !m.redoMemQuota.TryAcquire(requestMemSize) {
   731  				break LOOP
   732  			}
   733  
   734  			log.Debug("MemoryQuotaTracing: try acquire memory for redo log task",
   735  				zap.String("namespace", m.changefeedID.Namespace),
   736  				zap.String("changefeed", m.changefeedID.ID),
   737  				zap.Stringer("span", &tableSink.span),
   738  				zap.Uint64("memory", requestMemSize))
   739  
   740  			t := &redoTask{
   741  				span:          tableSink.span,
   742  				lowerBound:    lowerBound,
   743  				getUpperBound: m.getUpperBound,
   744  				tableSink:     tableSink,
   745  				callback: func(lastWrittenPos sorter.Position) {
   746  					p := &progress{
   747  						span:              tableSink.span,
   748  						nextLowerBoundPos: lastWrittenPos.Next(),
   749  						version:           slowestTableProgress.version,
   750  					}
   751  					m.redoProgressHeap.push(p)
   752  					select {
   753  					case m.redoWorkerAvailable <- struct{}{}:
   754  					default:
   755  					}
   756  				},
   757  				isCanceled: func() bool {
   758  					return tableSink.getState() != tablepb.TableStateReplicating
   759  				},
   760  			}
   761  			select {
   762  			case <-ctx.Done():
   763  				return ctx.Err()
   764  			case m.redoTaskChan <- t:
   765  				log.Debug("Generate redo task",
   766  					zap.String("namespace", m.changefeedID.Namespace),
   767  					zap.String("changefeed", m.changefeedID.ID),
   768  					zap.Stringer("span", &tableSink.span),
   769  					zap.Any("lowerBound", lowerBound),
   770  					zap.Any("currentUpperBound", upperBound),
   771  					zap.Float64("lag", time.Since(oracle.GetTimeFromTS(upperBound.CommitTs)).Seconds()))
   772  			default:
   773  				m.redoMemQuota.Refund(requestMemSize)
   774  				log.Debug("MemoryQuotaTracing: refund memory for redo log task",
   775  					zap.String("namespace", m.changefeedID.Namespace),
   776  					zap.String("changefeed", m.changefeedID.ID),
   777  					zap.Stringer("span", &tableSink.span),
   778  					zap.Uint64("memory", requestMemSize))
   779  				break LOOP
   780  			}
   781  		}
   782  		for ; i < len(progs); i++ {
   783  			m.redoProgressHeap.push(progs[i])
   784  		}
   785  		return nil
   786  	}
   787  
   788  	taskTicker := time.NewTicker(defaultGenerateTaskInterval)
   789  	defer taskTicker.Stop()
   790  	for {
   791  		select {
   792  		case <-ctx.Done():
   793  			return ctx.Err()
   794  		case <-taskTicker.C:
   795  			if err := dispatchTasks(); err != nil {
   796  				return errors.Trace(err)
   797  			}
   798  		case <-m.redoWorkerAvailable:
   799  			if err := dispatchTasks(); err != nil {
   800  				return errors.Trace(err)
   801  			}
   802  		}
   803  	}
   804  }
   805  
   806  // UpdateReceivedSorterResolvedTs updates the received sorter resolved ts for the table.
   807  // NOTE: it's still possible to be called during m.Close is in calling, so Close should
   808  // take care of this.
   809  func (m *SinkManager) UpdateReceivedSorterResolvedTs(span tablepb.Span, ts model.Ts) {
   810  	tableSink, ok := m.tableSinks.Load(span)
   811  	if !ok {
   812  		// It's possible that the table is in removing.
   813  		log.Debug("Table sink not found when updating resolved ts",
   814  			zap.String("namespace", m.changefeedID.Namespace),
   815  			zap.String("changefeed", m.changefeedID.ID),
   816  			zap.Stringer("span", &span))
   817  		return
   818  	}
   819  	tableSink.(*tableSinkWrapper).updateReceivedSorterResolvedTs(ts)
   820  }
   821  
   822  // UpdateBarrierTs update all tableSink's barrierTs in the SinkManager
   823  func (m *SinkManager) UpdateBarrierTs(globalBarrierTs model.Ts, tableBarrier map[model.TableID]model.Ts) {
   824  	m.tableSinks.Range(func(span tablepb.Span, value interface{}) bool {
   825  		barrierTs := globalBarrierTs
   826  		if tableBarrierTs, ok := tableBarrier[span.TableID]; ok && tableBarrierTs < globalBarrierTs {
   827  			barrierTs = tableBarrierTs
   828  		}
   829  		value.(*tableSinkWrapper).updateBarrierTs(barrierTs)
   830  		return true
   831  	})
   832  }
   833  
   834  // AddTable adds a table(TableSink) to the sink manager.
   835  func (m *SinkManager) AddTable(span tablepb.Span, startTs model.Ts, targetTs model.Ts) {
   836  	sinkWrapper := newTableSinkWrapper(
   837  		m.changefeedID,
   838  		span,
   839  		func() (s tablesink.TableSink, version uint64) {
   840  			if m.sinkFactory.TryLock() {
   841  				defer m.sinkFactory.Unlock()
   842  				if m.sinkFactory.f != nil {
   843  					s = m.sinkFactory.f.CreateTableSink(m.changefeedID, span, startTs, m.up.PDClock, m.metricsTableSinkTotalRows, m.metricsTableSinkFlushLagDuration)
   844  					version = m.sinkFactory.version
   845  				}
   846  			}
   847  			return
   848  		},
   849  		tablepb.TableStatePreparing,
   850  		startTs,
   851  		targetTs,
   852  		func(ctx context.Context) (model.Ts, error) {
   853  			return genReplicateTs(ctx, m.up.PDClient)
   854  		},
   855  	)
   856  
   857  	_, loaded := m.tableSinks.LoadOrStore(span, sinkWrapper)
   858  	if loaded {
   859  		log.Panic("Add an exists table sink",
   860  			zap.String("namespace", m.changefeedID.Namespace),
   861  			zap.String("changefeed", m.changefeedID.ID),
   862  			zap.Stringer("span", &span))
   863  		return
   864  	}
   865  	m.sinkMemQuota.AddTable(span)
   866  	m.redoMemQuota.AddTable(span)
   867  	log.Info("Add table sink",
   868  		zap.String("namespace", m.changefeedID.Namespace),
   869  		zap.String("changefeed", m.changefeedID.ID),
   870  		zap.Stringer("span", &span),
   871  		zap.Uint64("startTs", startTs),
   872  		zap.Uint64("version", sinkWrapper.version))
   873  }
   874  
   875  // StartTable sets the table(TableSink) state to replicating.
   876  func (m *SinkManager) StartTable(span tablepb.Span, startTs model.Ts) error {
   877  	log.Info("Start table sink",
   878  		zap.String("namespace", m.changefeedID.Namespace),
   879  		zap.String("changefeed", m.changefeedID.ID),
   880  		zap.Stringer("span", &span),
   881  		zap.Uint64("startTs", startTs),
   882  	)
   883  	tableSink, ok := m.tableSinks.Load(span)
   884  	if !ok {
   885  		log.Panic("Table sink not found when starting table stats",
   886  			zap.String("namespace", m.changefeedID.Namespace),
   887  			zap.String("changefeed", m.changefeedID.ID),
   888  			zap.Stringer("span", &span))
   889  	}
   890  
   891  	if err := tableSink.(*tableSinkWrapper).start(m.managerCtx, startTs); err != nil {
   892  		return err
   893  	}
   894  
   895  	m.sinkProgressHeap.push(&progress{
   896  		span:              span,
   897  		nextLowerBoundPos: sorter.Position{StartTs: 0, CommitTs: startTs + 1},
   898  		version:           tableSink.(*tableSinkWrapper).version,
   899  	})
   900  	if m.redoDMLMgr != nil {
   901  		m.redoProgressHeap.push(&progress{
   902  			span:              span,
   903  			nextLowerBoundPos: sorter.Position{StartTs: 0, CommitTs: startTs + 1},
   904  			version:           tableSink.(*tableSinkWrapper).version,
   905  		})
   906  	}
   907  	return nil
   908  }
   909  
   910  // AsyncStopTable sets the table(TableSink) state to stopped.
   911  func (m *SinkManager) AsyncStopTable(span tablepb.Span) bool {
   912  	tableSink, ok := m.tableSinks.Load(span)
   913  	if !ok {
   914  		// Just warn, because the table sink may be removed by another goroutine.
   915  		// This logic is the same as this function's caller.
   916  		log.Warn("Table sink not found when removing table",
   917  			zap.String("namespace", m.changefeedID.Namespace),
   918  			zap.String("changefeed", m.changefeedID.ID),
   919  			zap.Stringer("span", &span))
   920  	}
   921  	if tableSink.(*tableSinkWrapper).asyncStop() {
   922  		cleanedBytes := m.sinkMemQuota.RemoveTable(span)
   923  		cleanedBytes += m.redoMemQuota.RemoveTable(span)
   924  		log.Debug("MemoryQuotaTracing: Clean up memory quota for table sink task when removing table",
   925  			zap.String("namespace", m.changefeedID.Namespace),
   926  			zap.String("changefeed", m.changefeedID.ID),
   927  			zap.Stringer("span", &span),
   928  			zap.Uint64("memory", cleanedBytes))
   929  		return true
   930  	}
   931  	return false
   932  }
   933  
   934  // RemoveTable removes a table(TableSink) from the sink manager.
   935  func (m *SinkManager) RemoveTable(span tablepb.Span) {
   936  	// NOTICE: It is safe to only remove the table sink from the map.
   937  	// Because if we found the table sink is closed, we will not add it back to the heap.
   938  	// Also, no need to GC the SortEngine. Because the SortEngine also removes this table.
   939  	value, exists := m.tableSinks.LoadAndDelete(span)
   940  	if !exists {
   941  		log.Panic("Remove an unexist table sink",
   942  			zap.String("namespace", m.changefeedID.Namespace),
   943  			zap.String("changefeed", m.changefeedID.ID),
   944  			zap.Stringer("span", &span))
   945  	}
   946  	checkpointTs := value.(*tableSinkWrapper).getCheckpointTs()
   947  	log.Info("Remove table sink successfully",
   948  		zap.String("namespace", m.changefeedID.Namespace),
   949  		zap.String("changefeed", m.changefeedID.ID),
   950  		zap.Stringer("span", &span),
   951  		zap.Uint64("checkpointTs", checkpointTs.Ts))
   952  }
   953  
   954  // GetAllCurrentTableSpans returns all spans in the sinkManager.
   955  func (m *SinkManager) GetAllCurrentTableSpans() []tablepb.Span {
   956  	var spans []tablepb.Span
   957  	m.tableSinks.Range(func(key tablepb.Span, value interface{}) bool {
   958  		spans = append(spans, key)
   959  		return true
   960  	})
   961  	return spans
   962  }
   963  
   964  // GetAllCurrentTableSpansCount returns the table spans count in the sinkManager.
   965  func (m *SinkManager) GetAllCurrentTableSpansCount() int {
   966  	res := 0
   967  	m.tableSinks.Range(func(key tablepb.Span, value interface{}) bool {
   968  		res++
   969  		return true
   970  	})
   971  	return res
   972  }
   973  
   974  // GetTableState returns the table(TableSink) state.
   975  func (m *SinkManager) GetTableState(span tablepb.Span) (tablepb.TableState, bool) {
   976  	wrapper, ok := m.tableSinks.Load(span)
   977  	if !ok {
   978  		log.Debug("Table sink not found when getting table state",
   979  			zap.String("namespace", m.changefeedID.Namespace),
   980  			zap.String("changefeed", m.changefeedID.ID),
   981  			zap.Stringer("span", &span))
   982  		return tablepb.TableStateAbsent, false
   983  	}
   984  
   985  	// NOTE(qupeng): I'm not sure whether `SinkManager.AsyncStopTable` will be called
   986  	// again or not if it returns false. So we must retry `tableSink.asyncClose` here
   987  	// if necessary. It's better to remove the dirty logic in the future.
   988  	tableSink := wrapper.(*tableSinkWrapper)
   989  	if tableSink.getState() == tablepb.TableStateStopping && tableSink.asyncStop() {
   990  		cleanedBytes := m.sinkMemQuota.RemoveTable(span)
   991  		cleanedBytes += m.redoMemQuota.RemoveTable(span)
   992  		log.Debug("MemoryQuotaTracing: Clean up memory quota for table sink task when removing table",
   993  			zap.String("namespace", m.changefeedID.Namespace),
   994  			zap.String("changefeed", m.changefeedID.ID),
   995  			zap.Stringer("span", &span),
   996  			zap.Uint64("memory", cleanedBytes))
   997  	}
   998  	return tableSink.getState(), true
   999  }
  1000  
  1001  // GetTableStats returns the state of the table.
  1002  func (m *SinkManager) GetTableStats(span tablepb.Span) TableStats {
  1003  	value, ok := m.tableSinks.Load(span)
  1004  	if !ok {
  1005  		log.Panic("Table sink not found when getting table stats",
  1006  			zap.String("namespace", m.changefeedID.Namespace),
  1007  			zap.String("changefeed", m.changefeedID.ID),
  1008  			zap.Stringer("span", &span))
  1009  	}
  1010  	tableSink := value.(*tableSinkWrapper)
  1011  
  1012  	checkpointTs := tableSink.getCheckpointTs()
  1013  	lastSyncedTs := tableSink.getLastSyncedTs()
  1014  	m.sinkMemQuota.Release(span, checkpointTs)
  1015  	m.redoMemQuota.Release(span, checkpointTs)
  1016  
  1017  	advanceTimeoutInSec := util.GetOrZero(m.config.Sink.AdvanceTimeoutInSec)
  1018  	if advanceTimeoutInSec <= 0 {
  1019  		advanceTimeoutInSec = config.DefaultAdvanceTimeoutInSec
  1020  	}
  1021  	stuckCheck := time.Duration(advanceTimeoutInSec) * time.Second
  1022  
  1023  	if m.needsStuckCheck() {
  1024  		isStuck, sinkVersion := tableSink.sinkMaybeStuck(stuckCheck)
  1025  		if isStuck && m.putSinkFactoryError(errors.New("table sink stuck"), sinkVersion) {
  1026  			log.Warn("Table checkpoint is stuck too long, will restart the sink backend",
  1027  				zap.String("namespace", m.changefeedID.Namespace),
  1028  				zap.String("changefeed", m.changefeedID.ID),
  1029  				zap.Stringer("span", &span),
  1030  				zap.Any("checkpointTs", checkpointTs),
  1031  				zap.Float64("stuckCheck", stuckCheck.Seconds()),
  1032  				zap.Uint64("factoryVersion", sinkVersion))
  1033  		}
  1034  	}
  1035  
  1036  	var resolvedTs model.Ts
  1037  	// If redo log is enabled, we have to use redo log's resolved ts to calculate processor's min resolved ts.
  1038  	if m.redoDMLMgr != nil {
  1039  		resolvedTs = m.redoDMLMgr.GetResolvedTs(span)
  1040  	} else {
  1041  		resolvedTs = tableSink.getReceivedSorterResolvedTs()
  1042  	}
  1043  
  1044  	sinkUpperBound := tableSink.getUpperBoundTs()
  1045  	if sinkUpperBound < checkpointTs.ResolvedMark() {
  1046  		log.Panic("sinkManager: sink upperbound should not less than checkpoint ts",
  1047  			zap.String("namespace", m.changefeedID.Namespace),
  1048  			zap.String("changefeed", m.changefeedID.ID),
  1049  			zap.Stringer("span", &span),
  1050  			zap.Uint64("upperbound", sinkUpperBound),
  1051  			zap.Any("checkpointTs", checkpointTs))
  1052  	}
  1053  	return TableStats{
  1054  		CheckpointTs: checkpointTs.ResolvedMark(),
  1055  		ResolvedTs:   resolvedTs,
  1056  		LastSyncedTs: lastSyncedTs,
  1057  		BarrierTs:    tableSink.barrierTs.Load(),
  1058  	}
  1059  }
  1060  
  1061  // WaitForReady implements pkg/util.Runnable.
  1062  func (m *SinkManager) WaitForReady(ctx context.Context) {
  1063  	select {
  1064  	case <-ctx.Done():
  1065  	case <-m.ready:
  1066  	}
  1067  }
  1068  
  1069  // wait all sub-routines associated with `m.wg` returned.
  1070  func (m *SinkManager) waitSubroutines() {
  1071  	m.managerCancel()
  1072  	// Sink workers and redo workers can be blocked on MemQuota.BlockAcquire,
  1073  	// which doesn't watch m.managerCtx. So we must close these 2 MemQuotas
  1074  	// before wait them.
  1075  	m.sinkMemQuota.Close()
  1076  	m.redoMemQuota.Close()
  1077  	m.wg.Wait()
  1078  }
  1079  
  1080  // Close closes the manager. Must be called after `Run` returned.
  1081  func (m *SinkManager) Close() {
  1082  	log.Info("Closing sink manager",
  1083  		zap.String("namespace", m.changefeedID.Namespace),
  1084  		zap.String("changefeed", m.changefeedID.ID))
  1085  
  1086  	start := time.Now()
  1087  	m.waitSubroutines()
  1088  	// NOTE: It's unnecceary to close table sinks before clear sink factory.
  1089  	m.clearSinkFactory()
  1090  
  1091  	log.Info("Closed sink manager",
  1092  		zap.String("namespace", m.changefeedID.Namespace),
  1093  		zap.String("changefeed", m.changefeedID.ID),
  1094  		zap.Duration("cost", time.Since(start)))
  1095  }