github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/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  	"net/url"
    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/tiflow/cdc/async"
    29  	"github.com/pingcap/tiflow/cdc/entry"
    30  	"github.com/pingcap/tiflow/cdc/model"
    31  	"github.com/pingcap/tiflow/cdc/processor/sinkmanager"
    32  	"github.com/pingcap/tiflow/cdc/processor/sourcemanager"
    33  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    34  	"github.com/pingcap/tiflow/cdc/puller"
    35  	"github.com/pingcap/tiflow/cdc/redo"
    36  	"github.com/pingcap/tiflow/cdc/scheduler"
    37  	"github.com/pingcap/tiflow/cdc/scheduler/schedulepb"
    38  	"github.com/pingcap/tiflow/cdc/vars"
    39  	"github.com/pingcap/tiflow/pkg/config"
    40  	cerror "github.com/pingcap/tiflow/pkg/errors"
    41  	"github.com/pingcap/tiflow/pkg/etcd"
    42  	"github.com/pingcap/tiflow/pkg/filter"
    43  	"github.com/pingcap/tiflow/pkg/pdutil"
    44  	"github.com/pingcap/tiflow/pkg/retry"
    45  	"github.com/pingcap/tiflow/pkg/sink"
    46  	"github.com/pingcap/tiflow/pkg/upstream"
    47  	"github.com/pingcap/tiflow/pkg/util"
    48  	"github.com/prometheus/client_golang/prometheus"
    49  	"github.com/tikv/client-go/v2/oracle"
    50  	"go.uber.org/atomic"
    51  	"go.uber.org/zap"
    52  	"golang.org/x/sync/errgroup"
    53  )
    54  
    55  const (
    56  	backoffBaseDelayInMs = 5
    57  	maxTries             = 3
    58  )
    59  
    60  // Processor is the processor of changefeed data.
    61  type Processor interface {
    62  	// Tick is called periodically to drive the Processor's internal logic.
    63  	// The main logic of processor is in this function, including the calculation of many kinds of ts,
    64  	// maintain table components, error handling, etc.
    65  	//
    66  	// It can be called in etcd ticks, so it should never be blocked.
    67  	// Tick Returns: error and warnings. error will be propagated to the owner, and warnings will be record.
    68  	Tick(context.Context, *model.ChangeFeedInfo, *model.ChangeFeedStatus) (error, error)
    69  
    70  	// Close closes the processor.
    71  	Close() error
    72  }
    73  
    74  var _ Processor = (*processor)(nil)
    75  
    76  type processor struct {
    77  	changefeedID model.ChangeFeedID
    78  	captureInfo  *model.CaptureInfo
    79  	globalVars   *vars.GlobalVars
    80  
    81  	upstream     *upstream.Upstream
    82  	lastSchemaTs model.Ts
    83  
    84  	filter filter.Filter
    85  
    86  	// To manager DDL events and schema storage.
    87  	ddlHandler component[*ddlHandler]
    88  	// To manage MounterGroup.
    89  	mg component[entry.MounterGroup]
    90  	// To manage redo.DMLManager.
    91  	redo component[redo.DMLManager]
    92  
    93  	sourceManager component[*sourcemanager.SourceManager]
    94  
    95  	sinkManager component[*sinkmanager.SinkManager]
    96  
    97  	initialized *atomic.Bool
    98  	initializer *async.Initializer
    99  
   100  	lazyInit func(ctx context.Context) error
   101  	newAgent func(
   102  		context.Context, *model.Liveness, uint64, *config.SchedulerConfig,
   103  		etcd.OwnerCaptureInfoClient,
   104  	) (scheduler.Agent, error)
   105  	cfg *config.SchedulerConfig
   106  
   107  	liveness        *model.Liveness
   108  	agent           scheduler.Agent
   109  	changefeedEpoch uint64
   110  
   111  	// The latest changefeed info and status from meta storage. they are updated in every Tick.
   112  	// processor implements TableExecutor interface, so we need to add these two fields here to use them
   113  	// in `AddTableSpan` and `RemoveTableSpan`, otherwise we need to adjust the interface.
   114  	// we can refactor this step by step.
   115  	latestInfo   *model.ChangeFeedInfo
   116  	latestStatus *model.ChangeFeedStatus
   117  
   118  	ownerCaptureInfoClient etcd.OwnerCaptureInfoClient
   119  
   120  	metricSyncTableNumGauge      prometheus.Gauge
   121  	metricSchemaStorageGcTsGauge prometheus.Gauge
   122  	metricProcessorErrorCounter  prometheus.Counter
   123  	metricProcessorTickDuration  prometheus.Observer
   124  	metricsProcessorMemoryGauge  prometheus.Gauge
   125  }
   126  
   127  // checkReadyForMessages checks whether all necessary Etcd keys have been established.
   128  func (p *processor) checkReadyForMessages() bool {
   129  	return p.latestStatus != nil
   130  }
   131  
   132  var _ scheduler.TableExecutor = (*processor)(nil)
   133  
   134  // AddTableSpan implements TableExecutor interface.
   135  // AddTableSpan may cause by the following scenario
   136  // 1. `Create Table`, a new table dispatched to the processor, `isPrepare` should be false
   137  // 2. Prepare phase for 2 phase scheduling, `isPrepare` should be true.
   138  // 3. Replicating phase for 2 phase scheduling, `isPrepare` should be false
   139  func (p *processor) AddTableSpan(
   140  	ctx context.Context, span tablepb.Span, checkpoint tablepb.Checkpoint, isPrepare bool,
   141  ) (bool, error) {
   142  	if !p.checkReadyForMessages() {
   143  		return false, nil
   144  	}
   145  
   146  	failpoint.Inject("ProcessorAddTableError", func() {
   147  		failpoint.Return(false, cerror.New("processor add table injected error"))
   148  	})
   149  
   150  	startTs := checkpoint.CheckpointTs
   151  	if startTs == 0 {
   152  		log.Error("table start ts must not be 0",
   153  			zap.String("captureID", p.captureInfo.ID),
   154  			zap.String("namespace", p.changefeedID.Namespace),
   155  			zap.String("changefeed", p.changefeedID.ID),
   156  			zap.Stringer("span", &span),
   157  			zap.Uint64("checkpointTs", startTs),
   158  			zap.Bool("isPrepare", isPrepare))
   159  		return false, cerror.ErrUnexpected.FastGenByArgs("table start ts must not be 0")
   160  	}
   161  
   162  	state, alreadyExist := p.sinkManager.r.GetTableState(span)
   163  
   164  	if alreadyExist {
   165  		switch state {
   166  		// table is still `preparing`, which means the table is `replicating` on other captures.
   167  		// no matter `isPrepare` or not, just ignore it should be ok.
   168  		case tablepb.TableStatePreparing:
   169  			log.Warn("table is still preparing, ignore the request",
   170  				zap.String("captureID", p.captureInfo.ID),
   171  				zap.String("namespace", p.changefeedID.Namespace),
   172  				zap.String("changefeed", p.changefeedID.ID),
   173  				zap.Stringer("span", &span),
   174  				zap.Uint64("checkpointTs", startTs),
   175  				zap.Bool("isPrepare", isPrepare))
   176  			return true, nil
   177  		case tablepb.TableStatePrepared:
   178  			// table is `prepared`, and a `isPrepare = false` request indicate that old table should
   179  			// be stopped on original capture already, it's safe to start replicating data now.
   180  			if !isPrepare {
   181  				if p.redo.r.Enabled() {
   182  					// ResolvedTs is store in external storage when redo log is enabled, so we need to
   183  					// start table with ResolvedTs in redoDMLManager.
   184  					p.redo.r.StartTable(span, checkpoint.ResolvedTs)
   185  				}
   186  				if err := p.sinkManager.r.StartTable(span, startTs); err != nil {
   187  					return false, errors.Trace(err)
   188  				}
   189  			}
   190  			return true, nil
   191  		case tablepb.TableStateReplicating:
   192  			log.Warn("Ignore existing table",
   193  				zap.String("captureID", p.captureInfo.ID),
   194  				zap.String("namespace", p.changefeedID.Namespace),
   195  				zap.String("changefeed", p.changefeedID.ID),
   196  				zap.Stringer("span", &span),
   197  				zap.Uint64("checkpointTs", startTs),
   198  				zap.Bool("isPrepare", isPrepare))
   199  			return true, nil
   200  		case tablepb.TableStateStopped:
   201  			log.Warn("The same table exists but is stopped. Cancel it and continue.",
   202  				zap.String("captureID", p.captureInfo.ID),
   203  				zap.String("namespace", p.changefeedID.Namespace),
   204  				zap.String("changefeed", p.changefeedID.ID),
   205  				zap.Stringer("span", &span),
   206  				zap.Uint64("checkpointTs", startTs),
   207  				zap.Bool("isPrepare", isPrepare))
   208  			p.removeTable(span)
   209  		}
   210  	}
   211  
   212  	// table not found, can happen in 2 cases
   213  	// 1. this is a new table scheduling request, create the table and make it `replicating`
   214  	// 2. `prepare` phase for 2 phase scheduling, create the table and make it `preparing`
   215  	globalCheckpointTs := p.latestStatus.CheckpointTs
   216  	if startTs < globalCheckpointTs {
   217  		log.Warn("addTable: startTs < checkpoint",
   218  			zap.String("captureID", p.captureInfo.ID),
   219  			zap.String("namespace", p.changefeedID.Namespace),
   220  			zap.String("changefeed", p.changefeedID.ID),
   221  			zap.Stringer("span", &span),
   222  			zap.Uint64("checkpointTs", startTs),
   223  			zap.Bool("isPrepare", isPrepare))
   224  	}
   225  
   226  	p.sinkManager.r.AddTable(
   227  		span, startTs, p.latestInfo.TargetTs)
   228  	if p.redo.r.Enabled() {
   229  		p.redo.r.AddTable(span, startTs)
   230  	}
   231  	p.sourceManager.r.AddTable(span, p.getTableName(ctx, span.TableID), startTs)
   232  
   233  	return true, nil
   234  }
   235  
   236  // RemoveTableSpan implements TableExecutor interface.
   237  func (p *processor) RemoveTableSpan(span tablepb.Span) bool {
   238  	if !p.checkReadyForMessages() {
   239  		return false
   240  	}
   241  
   242  	_, exist := p.sinkManager.r.GetTableState(span)
   243  	if !exist {
   244  		log.Warn("Table which will be deleted is not found",
   245  			zap.String("capture", p.captureInfo.ID),
   246  			zap.String("namespace", p.changefeedID.Namespace),
   247  			zap.String("changefeed", p.changefeedID.ID),
   248  			zap.Stringer("span", &span))
   249  		return true
   250  	}
   251  	return p.sinkManager.r.AsyncStopTable(span)
   252  }
   253  
   254  // IsAddTableSpanFinished implements TableExecutor interface.
   255  func (p *processor) IsAddTableSpanFinished(span tablepb.Span, isPrepare bool) bool {
   256  	if !p.checkReadyForMessages() {
   257  		return false
   258  	}
   259  
   260  	globalCheckpointTs := p.latestStatus.CheckpointTs
   261  
   262  	var tableResolvedTs, tableCheckpointTs uint64
   263  	var state tablepb.TableState
   264  	done := func() bool {
   265  		var alreadyExist bool
   266  		state, alreadyExist = p.sinkManager.r.GetTableState(span)
   267  		if alreadyExist {
   268  			stats := p.sinkManager.r.GetTableStats(span)
   269  			tableResolvedTs = stats.ResolvedTs
   270  			tableCheckpointTs = stats.CheckpointTs
   271  		} else {
   272  			log.Panic("table which was added is not found",
   273  				zap.String("captureID", p.captureInfo.ID),
   274  				zap.String("namespace", p.changefeedID.Namespace),
   275  				zap.String("changefeed", p.changefeedID.ID),
   276  				zap.Stringer("span", &span),
   277  				zap.Bool("isPrepare", isPrepare))
   278  		}
   279  
   280  		if isPrepare {
   281  			return state == tablepb.TableStatePrepared
   282  		}
   283  
   284  		// The table is `replicating`, it's indicating that the `add table` must be finished.
   285  		return state == tablepb.TableStateReplicating
   286  	}
   287  	if !done() {
   288  		log.Debug("Add Table not finished",
   289  			zap.String("captureID", p.captureInfo.ID),
   290  			zap.String("namespace", p.changefeedID.Namespace),
   291  			zap.String("changefeed", p.changefeedID.ID),
   292  			zap.Stringer("span", &span),
   293  			zap.Uint64("tableResolvedTs", tableResolvedTs),
   294  			zap.Uint64("tableCheckpointTs", tableCheckpointTs),
   295  			zap.Uint64("globalCheckpointTs", globalCheckpointTs),
   296  			zap.Any("state", state),
   297  			zap.Bool("isPrepare", isPrepare))
   298  		return false
   299  	}
   300  
   301  	log.Info("Add Table finished",
   302  		zap.String("captureID", p.captureInfo.ID),
   303  		zap.String("namespace", p.changefeedID.Namespace),
   304  		zap.String("changefeed", p.changefeedID.ID),
   305  		zap.Stringer("span", &span),
   306  		zap.Uint64("tableResolvedTs", tableResolvedTs),
   307  		zap.Uint64("tableCheckpointTs", tableCheckpointTs),
   308  		zap.Uint64("globalCheckpointTs", globalCheckpointTs),
   309  		zap.Any("state", state),
   310  		zap.Bool("isPrepare", isPrepare))
   311  	return true
   312  }
   313  
   314  // IsRemoveTableSpanFinished implements TableExecutor interface.
   315  func (p *processor) IsRemoveTableSpanFinished(span tablepb.Span) (model.Ts, bool) {
   316  	if !p.checkReadyForMessages() {
   317  		return 0, false
   318  	}
   319  
   320  	state, ok := p.sinkManager.r.GetTableState(span)
   321  	if !ok {
   322  		log.Warn("table has been stopped",
   323  			zap.String("captureID", p.captureInfo.ID),
   324  			zap.String("namespace", p.changefeedID.Namespace),
   325  			zap.String("changefeed", p.changefeedID.ID),
   326  			zap.Stringer("span", &span))
   327  		return 0, true
   328  	}
   329  
   330  	stats := p.sinkManager.r.GetTableStats(span)
   331  	if state != tablepb.TableStateStopped {
   332  		log.Debug("table is still not stopped",
   333  			zap.String("captureID", p.captureInfo.ID),
   334  			zap.String("namespace", p.changefeedID.Namespace),
   335  			zap.String("changefeed", p.changefeedID.ID),
   336  			zap.Uint64("checkpointTs", stats.CheckpointTs),
   337  			zap.Stringer("span", &span),
   338  			zap.Any("tableStatus", state))
   339  		return 0, false
   340  	}
   341  
   342  	if p.redo.r.Enabled() {
   343  		p.redo.r.RemoveTable(span)
   344  	}
   345  	p.sinkManager.r.RemoveTable(span)
   346  	p.sourceManager.r.RemoveTable(span)
   347  	log.Info("table removed",
   348  		zap.String("captureID", p.captureInfo.ID),
   349  		zap.String("namespace", p.changefeedID.Namespace),
   350  		zap.String("changefeed", p.changefeedID.ID),
   351  		zap.Stringer("span", &span),
   352  		zap.Uint64("checkpointTs", stats.CheckpointTs))
   353  
   354  	return stats.CheckpointTs, true
   355  }
   356  
   357  // GetTableSpanStatus implements TableExecutor interface
   358  func (p *processor) GetTableSpanStatus(span tablepb.Span, collectStat bool) tablepb.TableStatus {
   359  	state, exist := p.sinkManager.r.GetTableState(span)
   360  	if !exist {
   361  		return tablepb.TableStatus{
   362  			TableID: span.TableID,
   363  			Span:    span,
   364  			State:   tablepb.TableStateAbsent,
   365  		}
   366  	}
   367  	sinkStats := p.sinkManager.r.GetTableStats(span)
   368  	stats := tablepb.Stats{}
   369  	if collectStat {
   370  		stats = p.getStatsFromSourceManagerAndSinkManager(span, sinkStats)
   371  	}
   372  	return tablepb.TableStatus{
   373  		TableID: span.TableID,
   374  		Span:    span,
   375  		Checkpoint: tablepb.Checkpoint{
   376  			CheckpointTs: sinkStats.CheckpointTs,
   377  			ResolvedTs:   sinkStats.ResolvedTs,
   378  			LastSyncedTs: sinkStats.LastSyncedTs,
   379  		},
   380  		State: state,
   381  		Stats: stats,
   382  	}
   383  }
   384  
   385  func (p *processor) getStatsFromSourceManagerAndSinkManager(
   386  	span tablepb.Span, sinkStats sinkmanager.TableStats,
   387  ) tablepb.Stats {
   388  	pullerStats := p.sourceManager.r.GetTablePullerStats(span)
   389  	now := p.upstream.PDClock.CurrentTime()
   390  
   391  	stats := tablepb.Stats{
   392  		RegionCount: pullerStats.RegionCount,
   393  		CurrentTs:   oracle.ComposeTS(oracle.GetPhysical(now), 0),
   394  		BarrierTs:   sinkStats.BarrierTs,
   395  		StageCheckpoints: map[string]tablepb.Checkpoint{
   396  			"puller-ingress": {
   397  				CheckpointTs: pullerStats.CheckpointTsIngress,
   398  				ResolvedTs:   pullerStats.ResolvedTsIngress,
   399  			},
   400  			"puller-egress": {
   401  				CheckpointTs: pullerStats.CheckpointTsEgress,
   402  				ResolvedTs:   pullerStats.ResolvedTsEgress,
   403  			},
   404  			"sink": {
   405  				CheckpointTs: sinkStats.CheckpointTs,
   406  				ResolvedTs:   sinkStats.ResolvedTs,
   407  			},
   408  		},
   409  	}
   410  
   411  	sortStats := p.sourceManager.r.GetTableSorterStats(span)
   412  	stats.StageCheckpoints["sorter-ingress"] = tablepb.Checkpoint{
   413  		CheckpointTs: sortStats.ReceivedMaxCommitTs,
   414  		ResolvedTs:   sortStats.ReceivedMaxResolvedTs,
   415  	}
   416  	stats.StageCheckpoints["sorter-egress"] = tablepb.Checkpoint{
   417  		CheckpointTs: sinkStats.ResolvedTs,
   418  		ResolvedTs:   sinkStats.ResolvedTs,
   419  	}
   420  
   421  	return stats
   422  }
   423  
   424  // NewProcessor creates a new processor
   425  func NewProcessor(
   426  	info *model.ChangeFeedInfo,
   427  	status *model.ChangeFeedStatus,
   428  	captureInfo *model.CaptureInfo,
   429  	changefeedID model.ChangeFeedID,
   430  	up *upstream.Upstream,
   431  	liveness *model.Liveness,
   432  	changefeedEpoch uint64,
   433  	cfg *config.SchedulerConfig,
   434  	ownerCaptureInfoClient etcd.OwnerCaptureInfoClient,
   435  	globalVars *vars.GlobalVars,
   436  ) *processor {
   437  	p := &processor{
   438  		upstream:        up,
   439  		changefeedID:    changefeedID,
   440  		captureInfo:     captureInfo,
   441  		liveness:        liveness,
   442  		changefeedEpoch: changefeedEpoch,
   443  		latestInfo:      info,
   444  		latestStatus:    status,
   445  
   446  		initialized: atomic.NewBool(false),
   447  
   448  		ownerCaptureInfoClient: ownerCaptureInfoClient,
   449  		globalVars:             globalVars,
   450  
   451  		metricSyncTableNumGauge: syncTableNumGauge.
   452  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   453  		metricProcessorErrorCounter: processorErrorCounter.
   454  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   455  		metricSchemaStorageGcTsGauge: processorSchemaStorageGcTsGauge.
   456  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   457  		metricProcessorTickDuration: processorTickDuration.
   458  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   459  		metricsProcessorMemoryGauge: processorMemoryGauge.
   460  			WithLabelValues(changefeedID.Namespace, changefeedID.ID),
   461  	}
   462  	p.lazyInit = p.lazyInitImpl
   463  	p.newAgent = p.newAgentImpl
   464  	p.cfg = cfg
   465  	p.initializer = async.NewInitializer()
   466  	return p
   467  }
   468  
   469  var processorIgnorableError = []*errors.Error{
   470  	cerror.ErrAdminStopProcessor,
   471  	cerror.ErrReactorFinished,
   472  }
   473  
   474  // isProcessorIgnorableError returns true if the error means the processor exits
   475  // normally, caused by changefeed pause, remove, etc.
   476  func isProcessorIgnorableError(err error) bool {
   477  	if err == nil {
   478  		return true
   479  	}
   480  	if errors.Cause(err) == context.Canceled {
   481  		return true
   482  	}
   483  	for _, e := range processorIgnorableError {
   484  		if e.Equal(err) {
   485  			return true
   486  		}
   487  	}
   488  	return false
   489  }
   490  
   491  // needPullerSafeModeAtStart returns true if the scheme is mysql compatible.
   492  // pullerSafeMode means to split all update kv entries whose commitTS
   493  // is older then the start time of this changefeed.
   494  func needPullerSafeModeAtStart(sinkURIStr string) (bool, error) {
   495  	sinkURI, err := url.Parse(sinkURIStr)
   496  	if err != nil {
   497  		return false, cerror.WrapError(cerror.ErrSinkURIInvalid, err)
   498  	}
   499  	scheme := sink.GetScheme(sinkURI)
   500  	return sink.IsMySQLCompatibleScheme(scheme), nil
   501  }
   502  
   503  // Tick implements the `orchestrator.State` interface
   504  // the `info` parameter is sent by metadata store, the `info` must be the latest value snapshot.
   505  // the `status` parameter is sent by metadata store, the `status` must be the latest value snapshot.
   506  // The main logic of processor is in this function, including the calculation of many kinds of ts,
   507  // maintain table components, error handling, etc.
   508  //
   509  // It can be called in etcd ticks, so it should never be blocked.
   510  func (p *processor) Tick(
   511  	ctx context.Context,
   512  	info *model.ChangeFeedInfo, status *model.ChangeFeedStatus,
   513  ) (error, error) {
   514  	if !p.initialized.Load() {
   515  		initialized, err := p.initializer.TryInitialize(ctx, p.lazyInit, p.globalVars.ChangefeedThreadPool)
   516  		if err != nil {
   517  			return errors.Trace(err), nil
   518  		}
   519  		if !initialized {
   520  			return nil, nil
   521  		}
   522  	}
   523  
   524  	p.latestInfo = info
   525  	p.latestStatus = status
   526  
   527  	// check upstream error first
   528  	if err := p.upstream.Error(); err != nil {
   529  		p.metricProcessorErrorCounter.Inc()
   530  		return err, nil
   531  	}
   532  	if p.upstream.IsClosed() {
   533  		log.Error("upstream is closed",
   534  			zap.Uint64("upstreamID", p.upstream.ID),
   535  			zap.String("namespace", p.changefeedID.Namespace),
   536  			zap.String("changefeed", p.changefeedID.ID))
   537  		return cerror.ErrUnexpected.FastGenByArgs("upstream is closed"), nil
   538  	}
   539  	// skip this tick
   540  	if !p.upstream.IsNormal() {
   541  		log.Warn("upstream is not ready, skip",
   542  			zap.Uint64("id", p.upstream.ID),
   543  			zap.Strings("pd", p.upstream.PdEndpoints),
   544  			zap.String("namespace", p.changefeedID.Namespace),
   545  			zap.String("changefeed", p.changefeedID.ID))
   546  		return nil, nil
   547  	}
   548  	startTime := time.Now()
   549  	err, warning := p.tick(ctx)
   550  	costTime := time.Since(startTime)
   551  	if costTime > processorLogsWarnDuration {
   552  		log.Warn("processor tick took too long",
   553  			zap.String("namespace", p.changefeedID.Namespace),
   554  			zap.String("changefeed", p.changefeedID.ID),
   555  			zap.String("capture", p.captureInfo.ID),
   556  			zap.Duration("duration", costTime))
   557  	}
   558  
   559  	p.metricProcessorTickDuration.Observe(costTime.Seconds())
   560  
   561  	// we should check if this error is nil,
   562  	// otherwise the function called below may panic.
   563  	if err == nil {
   564  		p.refreshMetrics()
   565  	} else {
   566  		p.metricProcessorErrorCounter.Inc()
   567  	}
   568  	return err, warning
   569  }
   570  
   571  func (p *processor) handleWarnings() error {
   572  	var err error
   573  	select {
   574  	case err = <-p.ddlHandler.warnings:
   575  	case err = <-p.mg.warnings:
   576  	case err = <-p.redo.warnings:
   577  	case err = <-p.sourceManager.warnings:
   578  	case err = <-p.sinkManager.warnings:
   579  	default:
   580  	}
   581  	return err
   582  }
   583  
   584  func (p *processor) tick(ctx context.Context) (error, error) {
   585  	warning := p.handleWarnings()
   586  	if err := p.handleErrorCh(); err != nil {
   587  		return errors.Trace(err), warning
   588  	}
   589  
   590  	barrier, err := p.agent.Tick(ctx)
   591  	if err != nil {
   592  		return errors.Trace(err), warning
   593  	}
   594  
   595  	if barrier != nil && barrier.GlobalBarrierTs != 0 {
   596  		p.updateBarrierTs(barrier)
   597  	}
   598  	p.doGCSchemaStorage()
   599  
   600  	return nil, warning
   601  }
   602  
   603  // lazyInitImpl create Filter, SchemaStorage, Mounter instances at the first tick.
   604  func (p *processor) lazyInitImpl(etcdCtx context.Context) (err error) {
   605  	if p.initialized.Load() {
   606  		return nil
   607  	}
   608  	// Here we use a separated context for sub-components, so we can custom the
   609  	// order of stopping all sub-components when closing the processor.
   610  	prcCtx := context.Background()
   611  
   612  	var tz *time.Location
   613  	// todo: get the timezone from the global config or the changefeed config?
   614  	tz, err = util.GetTimezone(config.GetGlobalServerConfig().TZ)
   615  	if err != nil {
   616  		return errors.Trace(err)
   617  	}
   618  
   619  	// Clone the config to avoid data race
   620  	cfConfig := p.latestInfo.Config.Clone()
   621  
   622  	p.filter, err = filter.NewFilter(cfConfig, util.GetTimeZoneName(tz))
   623  	if err != nil {
   624  		return errors.Trace(err)
   625  	}
   626  
   627  	if err = p.initDDLHandler(prcCtx); err != nil {
   628  		return err
   629  	}
   630  	p.ddlHandler.name = "ddlHandler"
   631  	p.ddlHandler.changefeedID = p.changefeedID
   632  	p.ddlHandler.spawn(prcCtx)
   633  
   634  	p.mg.r = entry.NewMounterGroup(p.ddlHandler.r.schemaStorage,
   635  		cfConfig.Mounter.WorkerNum,
   636  		p.filter, tz, p.changefeedID, cfConfig.Integrity)
   637  	p.mg.name = "MounterGroup"
   638  	p.mg.changefeedID = p.changefeedID
   639  	p.mg.spawn(prcCtx)
   640  
   641  	sourceID, err := pdutil.GetSourceID(prcCtx, p.upstream.PDClient)
   642  	if err != nil {
   643  		return errors.Trace(err)
   644  	}
   645  	log.Info("get sourceID from PD", zap.Uint64("sourceID", sourceID), zap.Stringer("changefeedID", p.changefeedID))
   646  	cfConfig.Sink.TiDBSourceID = sourceID
   647  
   648  	p.redo.r = redo.NewDMLManager(p.changefeedID, cfConfig.Consistent)
   649  	p.redo.name = "RedoManager"
   650  	p.redo.changefeedID = p.changefeedID
   651  	p.redo.spawn(prcCtx)
   652  
   653  	sortEngine, err := p.globalVars.SortEngineFactory.Create(p.changefeedID)
   654  	log.Info("Processor creates sort engine",
   655  		zap.String("namespace", p.changefeedID.Namespace),
   656  		zap.String("changefeed", p.changefeedID.ID),
   657  		zap.Error(err))
   658  	if err != nil {
   659  		return errors.Trace(err)
   660  	}
   661  
   662  	pullerSafeModeAtStart, err := needPullerSafeModeAtStart(p.latestInfo.SinkURI)
   663  	if err != nil {
   664  		return errors.Trace(err)
   665  	}
   666  	p.sourceManager.r = sourcemanager.New(
   667  		p.changefeedID, p.upstream, p.mg.r,
   668  		sortEngine, util.GetOrZero(cfConfig.BDRMode),
   669  		util.GetOrZero(cfConfig.EnableTableMonitor),
   670  		pullerSafeModeAtStart)
   671  	p.sourceManager.name = "SourceManager"
   672  	p.sourceManager.changefeedID = p.changefeedID
   673  	p.sourceManager.spawn(prcCtx)
   674  
   675  	p.sinkManager.r = sinkmanager.New(
   676  		p.changefeedID, p.latestInfo.SinkURI, cfConfig, p.upstream,
   677  		p.ddlHandler.r.schemaStorage, p.redo.r, p.sourceManager.r)
   678  	p.sinkManager.name = "SinkManager"
   679  	p.sinkManager.changefeedID = p.changefeedID
   680  	p.sinkManager.spawn(prcCtx)
   681  
   682  	// Bind them so that sourceManager can notify sinkManager.r.
   683  	p.sourceManager.r.OnResolve(p.sinkManager.r.UpdateReceivedSorterResolvedTs)
   684  	p.agent, err = p.newAgent(prcCtx, p.liveness, p.changefeedEpoch, p.cfg, p.ownerCaptureInfoClient)
   685  	if err != nil {
   686  		return err
   687  	}
   688  
   689  	p.initialized.Store(true)
   690  	log.Info("processor initialized",
   691  		zap.String("capture", p.captureInfo.ID),
   692  		zap.String("namespace", p.changefeedID.Namespace),
   693  		zap.String("changefeed", p.changefeedID.ID),
   694  		zap.Uint64("changefeedEpoch", p.changefeedEpoch))
   695  	return nil
   696  }
   697  
   698  func (p *processor) newAgentImpl(
   699  	ctx context.Context,
   700  	liveness *model.Liveness,
   701  	changefeedEpoch uint64,
   702  	cfg *config.SchedulerConfig,
   703  	client etcd.OwnerCaptureInfoClient,
   704  ) (ret scheduler.Agent, err error) {
   705  	messageServer := p.globalVars.MessageServer
   706  	messageRouter := p.globalVars.MessageRouter
   707  	captureID := p.globalVars.CaptureInfo.ID
   708  	ret, err = scheduler.NewAgent(
   709  		ctx, captureID, liveness,
   710  		messageServer, messageRouter, client, p, p.changefeedID,
   711  		changefeedEpoch, cfg)
   712  	return ret, errors.Trace(err)
   713  }
   714  
   715  // handleErrorCh listen the error channel and throw the error if it is not expected.
   716  func (p *processor) handleErrorCh() (err error) {
   717  	// TODO(qupeng): handle different errors in different ways.
   718  	select {
   719  	case err = <-p.ddlHandler.errors:
   720  	case err = <-p.mg.errors:
   721  	case err = <-p.redo.errors:
   722  	case err = <-p.sourceManager.errors:
   723  	case err = <-p.sinkManager.errors:
   724  	default:
   725  		return nil
   726  	}
   727  	if !isProcessorIgnorableError(err) {
   728  		log.Error("error on running processor",
   729  			zap.String("capture", p.captureInfo.ID),
   730  			zap.String("namespace", p.changefeedID.Namespace),
   731  			zap.String("changefeed", p.changefeedID.ID),
   732  			zap.Error(err))
   733  		return err
   734  	}
   735  	log.Info("processor exited",
   736  		zap.String("capture", p.captureInfo.ID),
   737  		zap.String("namespace", p.changefeedID.Namespace),
   738  		zap.String("changefeed", p.changefeedID.ID))
   739  	return cerror.ErrReactorFinished
   740  }
   741  
   742  func (p *processor) initDDLHandler(ctx context.Context) error {
   743  	checkpointTs := p.latestInfo.GetCheckpointTs(p.latestStatus)
   744  	minTableBarrierTs := p.latestStatus.MinTableBarrierTs
   745  	forceReplicate := p.latestInfo.Config.ForceReplicate
   746  
   747  	// if minTableBarrierTs == checkpointTs it means owner can't tell whether the DDL on checkpointTs has
   748  	// been executed or not. So the DDL puller must start at checkpointTs-1.
   749  	var ddlStartTs uint64
   750  	if minTableBarrierTs > checkpointTs {
   751  		ddlStartTs = checkpointTs
   752  	} else {
   753  		ddlStartTs = checkpointTs - 1
   754  	}
   755  
   756  	f, err := filter.NewFilter(p.latestInfo.Config, "")
   757  	if err != nil {
   758  		return errors.Trace(err)
   759  	}
   760  	schemaStorage, err := entry.NewSchemaStorage(p.upstream.KVStorage, ddlStartTs,
   761  		forceReplicate, p.changefeedID, util.RoleProcessor, f)
   762  	if err != nil {
   763  		return errors.Trace(err)
   764  	}
   765  
   766  	serverCfg := config.GetGlobalServerConfig()
   767  	changefeedID := model.DefaultChangeFeedID(p.changefeedID.ID + "_processor_ddl_puller")
   768  	ddlPuller := puller.NewDDLJobPuller(
   769  		ctx, p.upstream, ddlStartTs, serverCfg, changefeedID, schemaStorage, p.filter,
   770  	)
   771  	p.ddlHandler.r = &ddlHandler{puller: ddlPuller, schemaStorage: schemaStorage}
   772  	return nil
   773  }
   774  
   775  // updateBarrierTs updates barrierTs for all tables.
   776  func (p *processor) updateBarrierTs(barrier *schedulepb.Barrier) {
   777  	tableBarrier := p.calculateTableBarrierTs(barrier)
   778  	globalBarrierTs := barrier.GetGlobalBarrierTs()
   779  
   780  	schemaResolvedTs := p.ddlHandler.r.schemaStorage.ResolvedTs()
   781  	if schemaResolvedTs < globalBarrierTs {
   782  		// Do not update barrier ts that is larger than
   783  		// DDL puller's resolved ts.
   784  		// When DDL puller stall, resolved events that outputted by sorter
   785  		// may pile up in memory, as they have to wait DDL.
   786  		globalBarrierTs = schemaResolvedTs
   787  	}
   788  	log.Debug("update barrierTs",
   789  		zap.String("namespace", p.changefeedID.Namespace),
   790  		zap.String("changefeed", p.changefeedID.ID),
   791  		zap.Any("tableBarriers", barrier.GetTableBarriers()),
   792  		zap.Uint64("globalBarrierTs", globalBarrierTs))
   793  
   794  	p.sinkManager.r.UpdateBarrierTs(globalBarrierTs, tableBarrier)
   795  }
   796  
   797  func (p *processor) getTableName(ctx context.Context, tableID model.TableID) string {
   798  	// FIXME: using GetLastSnapshot here would be confused and get the wrong table name
   799  	// after `rename table` DDL, since `rename table` keeps the tableID unchanged
   800  	var tableName *model.TableName
   801  	retry.Do(ctx, func() error { //nolint:errcheck
   802  		if x, ok := p.ddlHandler.r.schemaStorage.GetLastSnapshot().PhysicalTableByID(tableID); ok {
   803  			tableName = &x.TableName
   804  			return nil
   805  		}
   806  		return errors.Errorf("failed to get table name, fallback to use table id: %d",
   807  			tableID)
   808  	}, retry.WithBackoffBaseDelay(backoffBaseDelayInMs),
   809  		retry.WithMaxTries(maxTries),
   810  		retry.WithIsRetryableErr(cerror.IsRetryableError))
   811  
   812  	if tableName == nil {
   813  		log.Warn("failed to get table name for metric",
   814  			zap.String("namespace", p.changefeedID.Namespace),
   815  			zap.String("changefeed", p.changefeedID.ID),
   816  			zap.Any("tableID", tableID))
   817  		return strconv.Itoa(int(tableID))
   818  	}
   819  
   820  	return tableName.QuoteString()
   821  }
   822  
   823  func (p *processor) removeTable(span tablepb.Span) {
   824  	if p.redo.r.Enabled() {
   825  		p.redo.r.RemoveTable(span)
   826  	}
   827  	p.sinkManager.r.RemoveTable(span)
   828  	p.sourceManager.r.RemoveTable(span)
   829  }
   830  
   831  // doGCSchemaStorage trigger the schema storage GC
   832  func (p *processor) doGCSchemaStorage() {
   833  	if p.ddlHandler.r.schemaStorage == nil {
   834  		// schemaStorage is nil only in test
   835  		return
   836  	}
   837  
   838  	if p.latestStatus == nil {
   839  		// This could happen if Etcd data is not complete.
   840  		return
   841  	}
   842  
   843  	// Please refer to `unmarshalAndMountRowChanged` in cdc/entry/mounter.go
   844  	// for why we need -1.
   845  	lastSchemaTs := p.ddlHandler.r.schemaStorage.DoGC(p.latestStatus.CheckpointTs - 1)
   846  	if p.lastSchemaTs == lastSchemaTs {
   847  		return
   848  	}
   849  	p.lastSchemaTs = lastSchemaTs
   850  
   851  	log.Debug("finished gc in schema storage",
   852  		zap.Uint64("gcTs", lastSchemaTs),
   853  		zap.String("namespace", p.changefeedID.Namespace),
   854  		zap.String("changefeed", p.changefeedID.ID))
   855  	lastSchemaPhysicalTs := oracle.ExtractPhysical(lastSchemaTs)
   856  	p.metricSchemaStorageGcTsGauge.Set(float64(lastSchemaPhysicalTs))
   857  }
   858  
   859  func (p *processor) refreshMetrics() {
   860  	// Before the processor is initialized, we should not refresh metrics.
   861  	// Otherwise, it will cause panic.
   862  	if !p.initialized.Load() {
   863  		return
   864  	}
   865  	p.metricSyncTableNumGauge.Set(float64(p.sinkManager.r.GetAllCurrentTableSpansCount()))
   866  }
   867  
   868  // Close closes the processor. It must be called explicitly to stop all sub-components.
   869  func (p *processor) Close() error {
   870  	log.Info("processor closing ...",
   871  		zap.String("namespace", p.changefeedID.Namespace),
   872  		zap.String("changefeed", p.changefeedID.ID))
   873  	p.initializer.Terminate()
   874  	// clean up metrics first to avoid some metrics are not cleaned up
   875  	// when error occurs during closing the processor
   876  	p.cleanupMetrics()
   877  
   878  	p.sinkManager.stop()
   879  	p.sinkManager.r = nil
   880  	p.sourceManager.stop()
   881  	p.sourceManager.r = nil
   882  	p.redo.stop()
   883  	p.mg.stop()
   884  	p.ddlHandler.stop()
   885  
   886  	if p.globalVars != nil && p.globalVars.SortEngineFactory != nil {
   887  		if err := p.globalVars.SortEngineFactory.Drop(p.changefeedID); err != nil {
   888  			log.Error("Processor drop event sort engine fail",
   889  				zap.String("namespace", p.changefeedID.Namespace),
   890  				zap.String("changefeed", p.changefeedID.ID),
   891  				zap.Error(err))
   892  			return errors.Trace(err)
   893  		}
   894  		log.Info("Processor drop sort engine successfully",
   895  			zap.String("namespace", p.changefeedID.Namespace),
   896  			zap.String("changefeed", p.changefeedID.ID))
   897  	}
   898  
   899  	if p.agent != nil {
   900  		log.Info("Processor try to close agent",
   901  			zap.String("namespace", p.changefeedID.Namespace),
   902  			zap.String("changefeed", p.changefeedID.ID))
   903  		if err := p.agent.Close(); err != nil {
   904  			log.Warn("close agent meet error",
   905  				zap.String("namespace", p.changefeedID.Namespace),
   906  				zap.String("changefeed", p.changefeedID.ID),
   907  				zap.Error(err))
   908  		}
   909  		log.Info("Processor closed agent successfully",
   910  			zap.String("namespace", p.changefeedID.Namespace),
   911  			zap.String("changefeed", p.changefeedID.ID))
   912  		p.agent = nil
   913  	}
   914  
   915  	// mark tables share the same ctx with its original table, don't need to cancel
   916  	failpoint.Inject("processorStopDelay", nil)
   917  
   918  	log.Info("processor closed",
   919  		zap.String("namespace", p.changefeedID.Namespace),
   920  		zap.String("changefeed", p.changefeedID.ID))
   921  
   922  	return nil
   923  }
   924  
   925  func (p *processor) cleanupMetrics() {
   926  	syncTableNumGauge.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID)
   927  	processorErrorCounter.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID)
   928  	processorSchemaStorageGcTsGauge.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID)
   929  	processorTickDuration.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID)
   930  	processorMemoryGauge.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID)
   931  
   932  	ok := puller.PullerEventCounter.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID, "kv")
   933  	if !ok {
   934  		log.Warn("delete puller event counter metrics failed",
   935  			zap.String("namespace", p.changefeedID.Namespace),
   936  			zap.String("changefeed", p.changefeedID.ID),
   937  			zap.String("type", "kv"))
   938  	}
   939  	ok = puller.PullerEventCounter.DeleteLabelValues(p.changefeedID.Namespace, p.changefeedID.ID, "resolved")
   940  	if !ok {
   941  		log.Warn("delete puller event counter metrics failed",
   942  			zap.String("namespace", p.changefeedID.Namespace),
   943  			zap.String("changefeed", p.changefeedID.ID),
   944  			zap.String("type", "resolved"))
   945  	}
   946  }
   947  
   948  // WriteDebugInfo write the debug info to Writer
   949  func (p *processor) WriteDebugInfo(w io.Writer) error {
   950  	if !p.initialized.Load() {
   951  		fmt.Fprintln(w, "processor is not initialized")
   952  		return nil
   953  	}
   954  	fmt.Fprintf(w, "%+v\n%+v\n", *p.latestInfo, *p.latestStatus)
   955  	spans := p.sinkManager.r.GetAllCurrentTableSpans()
   956  	for _, span := range spans {
   957  		state, _ := p.sinkManager.r.GetTableState(span)
   958  		stats := p.sinkManager.r.GetTableStats(span)
   959  		// TODO: add table name.
   960  		fmt.Fprintf(w, "span: %s, resolvedTs: %d, checkpointTs: %d, state: %s\n",
   961  			&span, stats.ResolvedTs, stats.CheckpointTs, state)
   962  	}
   963  
   964  	return nil
   965  }
   966  
   967  func (p *processor) calculateTableBarrierTs(
   968  	barrier *schedulepb.Barrier,
   969  ) map[model.TableID]model.Ts {
   970  	tableBarrierTs := make(map[model.TableID]model.Ts)
   971  	for _, tb := range barrier.TableBarriers {
   972  		tableBarrierTs[tb.TableID] = tb.BarrierTs
   973  	}
   974  	return tableBarrierTs
   975  }
   976  
   977  type component[R util.Runnable] struct {
   978  	r            R
   979  	name         string
   980  	ctx          context.Context
   981  	cancel       context.CancelFunc
   982  	errors       chan error
   983  	warnings     chan error
   984  	wg           sync.WaitGroup
   985  	changefeedID model.ChangeFeedID
   986  }
   987  
   988  func (c *component[R]) spawn(ctx context.Context) {
   989  	c.ctx, c.cancel = context.WithCancel(ctx)
   990  	c.errors = make(chan error, 16)
   991  	c.warnings = make(chan error, 16)
   992  
   993  	changefeedID := c.changefeedID
   994  	c.wg.Add(1)
   995  	go func() {
   996  		defer c.wg.Done()
   997  		err := c.r.Run(c.ctx, c.warnings)
   998  		if err != nil && errors.Cause(err) != context.Canceled {
   999  			log.Error("processor sub-component fails",
  1000  				zap.String("namespace", changefeedID.Namespace),
  1001  				zap.String("changefeed", changefeedID.ID),
  1002  				zap.String("name", c.name),
  1003  				zap.Error(err))
  1004  			select {
  1005  			case <-c.ctx.Done():
  1006  			case c.errors <- err:
  1007  			}
  1008  		}
  1009  	}()
  1010  	c.r.WaitForReady(ctx)
  1011  	log.Info("processor sub-component starts",
  1012  		zap.String("namespace", changefeedID.Namespace),
  1013  		zap.String("changefeed", changefeedID.ID),
  1014  		zap.String("name", c.name))
  1015  }
  1016  
  1017  func (c *component[R]) stop() {
  1018  	if c.cancel == nil {
  1019  		log.Info("processor sub-component isn't started",
  1020  			zap.String("namespace", c.changefeedID.Namespace),
  1021  			zap.String("changefeed", c.changefeedID.ID),
  1022  			zap.String("name", c.name))
  1023  		return
  1024  	}
  1025  	log.Info("processor sub-component is in stopping",
  1026  		zap.String("namespace", c.changefeedID.Namespace),
  1027  		zap.String("changefeed", c.changefeedID.ID),
  1028  		zap.String("name", c.name))
  1029  	c.cancel()
  1030  	c.wg.Wait()
  1031  	c.r.Close()
  1032  }
  1033  
  1034  type ddlHandler struct {
  1035  	puller        puller.DDLJobPuller
  1036  	schemaStorage entry.SchemaStorage
  1037  }
  1038  
  1039  func (d *ddlHandler) Run(ctx context.Context, _ ...chan<- error) error {
  1040  	g, ctx := errgroup.WithContext(ctx)
  1041  	// d.puller will update the schemaStorage.
  1042  	g.Go(func() error { return d.puller.Run(ctx) })
  1043  	g.Go(func() error {
  1044  		for {
  1045  			var jobEntry *model.DDLJobEntry
  1046  			select {
  1047  			case <-ctx.Done():
  1048  				return nil
  1049  			case jobEntry = <-d.puller.Output():
  1050  			}
  1051  			failpoint.Inject("processorDDLResolved", nil)
  1052  			if jobEntry.OpType == model.OpTypeResolved {
  1053  				d.schemaStorage.AdvanceResolvedTs(jobEntry.CRTs)
  1054  			}
  1055  		}
  1056  	})
  1057  	return g.Wait()
  1058  }
  1059  
  1060  func (d *ddlHandler) WaitForReady(_ context.Context) {}
  1061  
  1062  func (d *ddlHandler) Close() {}