github.com/pingcap/tiflow@v0.0.0-20240520035814-5bf52d54e205/cdc/processor/processor_test.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  	"encoding/json"
    19  	"fmt"
    20  	"math"
    21  	"os"
    22  	"sync/atomic"
    23  	"testing"
    24  
    25  	"github.com/pingcap/errors"
    26  	"github.com/pingcap/failpoint"
    27  	"github.com/pingcap/tiflow/cdc/async"
    28  	"github.com/pingcap/tiflow/cdc/entry"
    29  	"github.com/pingcap/tiflow/cdc/entry/schema"
    30  	"github.com/pingcap/tiflow/cdc/model"
    31  	"github.com/pingcap/tiflow/cdc/processor/sinkmanager"
    32  	"github.com/pingcap/tiflow/cdc/processor/tablepb"
    33  	"github.com/pingcap/tiflow/cdc/redo"
    34  	"github.com/pingcap/tiflow/cdc/scheduler"
    35  	"github.com/pingcap/tiflow/cdc/scheduler/schedulepb"
    36  	"github.com/pingcap/tiflow/cdc/vars"
    37  	"github.com/pingcap/tiflow/pkg/config"
    38  	cerror "github.com/pingcap/tiflow/pkg/errors"
    39  	"github.com/pingcap/tiflow/pkg/etcd"
    40  	"github.com/pingcap/tiflow/pkg/orchestrator"
    41  	redoPkg "github.com/pingcap/tiflow/pkg/redo"
    42  	"github.com/pingcap/tiflow/pkg/spanz"
    43  	"github.com/pingcap/tiflow/pkg/upstream"
    44  	"github.com/stretchr/testify/require"
    45  )
    46  
    47  func newProcessor4Test(
    48  	t *testing.T,
    49  	info *model.ChangeFeedInfo,
    50  	status *model.ChangeFeedStatus,
    51  	captureInfo *model.CaptureInfo,
    52  	liveness *model.Liveness,
    53  	cfg *config.SchedulerConfig,
    54  	enableRedo bool,
    55  	client etcd.OwnerCaptureInfoClient,
    56  	globalVars *vars.GlobalVars,
    57  ) *processor {
    58  	changefeedID := model.ChangeFeedID4Test("processor-test", "processor-test")
    59  	up := upstream.NewUpstream4Test(&sinkmanager.MockPD{})
    60  	p := NewProcessor(
    61  		info,
    62  		status,
    63  		captureInfo,
    64  		changefeedID, up, liveness, 0, cfg, client, globalVars)
    65  	// Some cases want to send errors to the processor without initializing it.
    66  	p.sinkManager.errors = make(chan error, 16)
    67  	p.lazyInit = func(ctx context.Context) error {
    68  		if p.initialized.Load() {
    69  			return nil
    70  		}
    71  
    72  		if !enableRedo {
    73  			p.redo.r = redo.NewDisabledDMLManager()
    74  		} else {
    75  			tmpDir := t.TempDir()
    76  			redoDir := fmt.Sprintf("%s/%s", tmpDir, changefeedID)
    77  			dmlMgr := redo.NewDMLManager(changefeedID, &config.ConsistentConfig{
    78  				Level:                 string(redoPkg.ConsistentLevelEventual),
    79  				MaxLogSize:            redoPkg.DefaultMaxLogSize,
    80  				FlushIntervalInMs:     redoPkg.DefaultFlushIntervalInMs,
    81  				MetaFlushIntervalInMs: redoPkg.DefaultMetaFlushIntervalInMs,
    82  				EncodingWorkerNum:     redoPkg.DefaultEncodingWorkerNum,
    83  				FlushWorkerNum:        redoPkg.DefaultFlushWorkerNum,
    84  				Storage:               "file://" + redoDir,
    85  				UseFileBackend:        false,
    86  			})
    87  			p.redo.r = dmlMgr
    88  		}
    89  		p.redo.name = "RedoManager"
    90  		p.redo.changefeedID = changefeedID
    91  		p.redo.spawn(ctx)
    92  
    93  		p.agent = &mockAgent{executor: p, liveness: liveness}
    94  		p.sinkManager.r, p.sourceManager.r, _ = sinkmanager.NewManagerWithMemEngine(
    95  			t, changefeedID, info, p.redo.r)
    96  		p.sinkManager.name = "SinkManager"
    97  		p.sinkManager.changefeedID = changefeedID
    98  		p.sinkManager.spawn(ctx)
    99  		p.sourceManager.name = "SourceManager"
   100  		p.sourceManager.changefeedID = changefeedID
   101  		p.sourceManager.spawn(ctx)
   102  
   103  		// NOTICE: we have to bind the sourceManager to the sinkManager
   104  		// otherwise the sinkManager will not receive the resolvedTs.
   105  		p.sourceManager.r.OnResolve(p.sinkManager.r.UpdateReceivedSorterResolvedTs)
   106  
   107  		p.initialized.Store(true)
   108  		return nil
   109  	}
   110  	p.initializer = async.NewInitializer()
   111  
   112  	p.ddlHandler.r = &ddlHandler{
   113  		schemaStorage: &mockSchemaStorage{t: t, resolvedTs: math.MaxUint64},
   114  	}
   115  	return p
   116  }
   117  
   118  // nolint
   119  func initProcessor4Test(t *testing.T, liveness *model.Liveness, enableRedo bool,
   120  	globalVars *vars.GlobalVars, changefeedVars *model.ChangeFeedInfo,
   121  ) (*processor, *orchestrator.ReactorStateTester, *orchestrator.ChangefeedReactorState) {
   122  	changefeedInfo := `
   123  {
   124      "sink-uri": "blackhole://",
   125      "create-time": "2020-02-02T00:00:00.000000+00:00",
   126      "start-ts": 0,
   127      "target-ts": 0,
   128      "admin-job-type": 0,
   129      "sort-engine": "memory",
   130      "sort-dir": ".",
   131      "config": {
   132          "case-sensitive": true,
   133          "force-replicate": false,
   134          "check-gc-safe-point": true,
   135          "filter": {
   136              "rules": [
   137                  "*.*"
   138              ],
   139              "ignore-txn-start-ts": null
   140          },
   141          "mounter": {
   142              "worker-num": 16
   143          },
   144          "sink": {
   145              "dispatchers": null,
   146              "protocol": "open-protocol",
   147              "advance-timeout-in-sec": 150
   148          }
   149      },
   150      "state": "normal",
   151      "history": null,
   152      "error": null,
   153      "sync-point-enabled": false,
   154      "sync-point-interval": 600000000000
   155  }
   156  `
   157  	changefeed := orchestrator.NewChangefeedReactorState(
   158  		etcd.DefaultCDCClusterID, model.DefaultChangeFeedID(changefeedVars.ID))
   159  	captureInfo := &model.CaptureInfo{ID: "capture-test", AdvertiseAddr: "127.0.0.1:0000"}
   160  	cfg := config.NewDefaultSchedulerConfig()
   161  
   162  	captureID := globalVars.CaptureInfo.ID
   163  	changefeedID := changefeedVars.ID
   164  	tester := orchestrator.NewReactorStateTester(t, changefeed, map[string]string{
   165  		fmt.Sprintf("%s/capture/%s",
   166  			etcd.DefaultClusterAndMetaPrefix,
   167  			captureID): `{"id":"` + captureID + `","address":"127.0.0.1:8300"}`,
   168  		fmt.Sprintf("%s/changefeed/info/%s",
   169  			etcd.DefaultClusterAndNamespacePrefix,
   170  			changefeedID): changefeedInfo,
   171  		fmt.Sprintf("%s/changefeed/status/%s",
   172  			etcd.DefaultClusterAndNamespacePrefix,
   173  			changefeedVars.ID): `{"resolved-ts":0,"checkpoint-ts":0,"admin-job-type":0}`,
   174  	})
   175  	p := newProcessor4Test(t, changefeed.Info, changefeed.Status, captureInfo, liveness, cfg, enableRedo, nil, globalVars)
   176  
   177  	return p, tester, changefeed
   178  }
   179  
   180  type mockSchemaStorage struct {
   181  	// dummy to provide default versions of unimplemented interface methods,
   182  	// as we only need ResolvedTs() and DoGC() in unit tests.
   183  	entry.SchemaStorage
   184  
   185  	t          *testing.T
   186  	lastGcTs   uint64
   187  	resolvedTs uint64
   188  }
   189  
   190  func (s *mockSchemaStorage) ResolvedTs() uint64 {
   191  	return s.resolvedTs
   192  }
   193  
   194  func (s *mockSchemaStorage) DoGC(ts uint64) uint64 {
   195  	require.LessOrEqual(s.t, s.lastGcTs, ts)
   196  	atomic.StoreUint64(&s.lastGcTs, ts)
   197  	return ts
   198  }
   199  
   200  func (s *mockSchemaStorage) GetLastSnapshot() *schema.Snapshot {
   201  	return schema.NewEmptySnapshot(false)
   202  }
   203  
   204  type mockAgent struct {
   205  	// dummy to satisfy the interface
   206  	scheduler.Agent
   207  
   208  	executor scheduler.TableExecutor
   209  	liveness *model.Liveness
   210  	isClosed bool
   211  }
   212  
   213  func (a *mockAgent) Tick(_ context.Context) (*schedulepb.Barrier, error) {
   214  	return nil, nil
   215  }
   216  
   217  func (a *mockAgent) Close() error {
   218  	a.isClosed = true
   219  	return nil
   220  }
   221  
   222  func TestTableExecutorAddingTableIndirectly(t *testing.T) {
   223  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   224  	ctx := context.Background()
   225  	liveness := model.LivenessCaptureAlive
   226  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   227  
   228  	// init tick
   229  	checkChangefeedNormal(changefeed)
   230  	require.Nil(t, p.lazyInit(ctx))
   231  	createTaskPosition(changefeed, p.captureInfo)
   232  	tester.MustApplyPatches()
   233  	changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   234  		status.CheckpointTs = 20
   235  		return status, true, nil
   236  	})
   237  	tester.MustApplyPatches()
   238  
   239  	// no operation
   240  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   241  	require.Nil(t, err)
   242  	tester.MustApplyPatches()
   243  
   244  	// table-1: `preparing` -> `prepared` -> `replicating`
   245  	span := spanz.TableIDToComparableSpan(1)
   246  	ok, err := p.AddTableSpan(ctx, span, tablepb.Checkpoint{CheckpointTs: 20}, true)
   247  	require.NoError(t, err)
   248  	require.True(t, ok)
   249  	p.sinkManager.r.UpdateBarrierTs(20, nil)
   250  	stats := p.sinkManager.r.GetTableStats(span)
   251  	require.Equal(t, model.Ts(20), stats.CheckpointTs)
   252  	require.Equal(t, model.Ts(20), stats.ResolvedTs)
   253  	require.Equal(t, model.Ts(20), stats.BarrierTs)
   254  	require.Len(t, p.sinkManager.r.GetAllCurrentTableSpans(), 1)
   255  	require.Equal(t, 1, p.sinkManager.r.GetAllCurrentTableSpansCount())
   256  
   257  	done := p.IsAddTableSpanFinished(spanz.TableIDToComparableSpan(1), true)
   258  	require.False(t, done)
   259  	state, ok := p.sinkManager.r.GetTableState(span)
   260  	require.True(t, ok)
   261  	require.Equal(t, tablepb.TableStatePreparing, state)
   262  
   263  	// Push the resolved ts, mock that sorterNode receive first resolved event.
   264  	p.sourceManager.r.Add(
   265  		span,
   266  		[]*model.PolymorphicEvent{{
   267  			CRTs: 101,
   268  			RawKV: &model.RawKVEntry{
   269  				OpType: model.OpTypeResolved,
   270  				CRTs:   101,
   271  			},
   272  		}}...,
   273  	)
   274  
   275  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   276  	require.Nil(t, err)
   277  	tester.MustApplyPatches()
   278  
   279  	done = p.IsAddTableSpanFinished(span, true)
   280  	require.True(t, done)
   281  	state, ok = p.sinkManager.r.GetTableState(span)
   282  	require.True(t, ok)
   283  	require.Equal(t, tablepb.TableStatePrepared, state)
   284  
   285  	ok, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 30}, true)
   286  	require.NoError(t, err)
   287  	require.True(t, ok)
   288  	stats = p.sinkManager.r.GetTableStats(span)
   289  	require.Equal(t, model.Ts(20), stats.CheckpointTs)
   290  	require.Equal(t, model.Ts(101), stats.ResolvedTs)
   291  	require.Equal(t, model.Ts(20), stats.BarrierTs)
   292  
   293  	// Start to replicate table-1.
   294  	ok, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 30}, false)
   295  	require.NoError(t, err)
   296  	require.True(t, ok)
   297  
   298  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   299  	require.Nil(t, err)
   300  	tester.MustApplyPatches()
   301  
   302  	// table-1: `prepared` -> `replicating`
   303  	state, ok = p.sinkManager.r.GetTableState(span)
   304  	require.True(t, ok)
   305  	require.Equal(t, tablepb.TableStateReplicating, state)
   306  
   307  	err = p.Close()
   308  	require.Nil(t, err)
   309  	require.Nil(t, p.agent)
   310  }
   311  
   312  func TestTableExecutorAddingTableIndirectlyWithRedoEnabled(t *testing.T) {
   313  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   314  	ctx := context.Background()
   315  	liveness := model.LivenessCaptureAlive
   316  	p, tester, changefeed := initProcessor4Test(t, &liveness, true, globalVars, changefeedVars)
   317  
   318  	// init tick
   319  	checkChangefeedNormal(changefeed)
   320  	require.Nil(t, p.lazyInit(ctx))
   321  	createTaskPosition(changefeed, p.captureInfo)
   322  	tester.MustApplyPatches()
   323  	changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   324  		status.CheckpointTs = 20
   325  		return status, true, nil
   326  	})
   327  	tester.MustApplyPatches()
   328  
   329  	// no operation
   330  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   331  	require.Nil(t, err)
   332  	tester.MustApplyPatches()
   333  
   334  	// table-1: `preparing` -> `prepared` -> `replicating`
   335  	span := spanz.TableIDToComparableSpan(1)
   336  	ok, err := p.AddTableSpan(ctx, span, tablepb.Checkpoint{CheckpointTs: 20}, true)
   337  	require.NoError(t, err)
   338  	require.True(t, ok)
   339  	p.sinkManager.r.UpdateBarrierTs(20, nil)
   340  	stats := p.sinkManager.r.GetTableStats(span)
   341  	require.Equal(t, model.Ts(20), stats.CheckpointTs)
   342  	require.Equal(t, model.Ts(20), stats.ResolvedTs)
   343  	require.Equal(t, model.Ts(20), stats.BarrierTs)
   344  	require.Len(t, p.sinkManager.r.GetAllCurrentTableSpans(), 1)
   345  	require.Equal(t, 1, p.sinkManager.r.GetAllCurrentTableSpansCount())
   346  
   347  	done := p.IsAddTableSpanFinished(spanz.TableIDToComparableSpan(1), true)
   348  	require.False(t, done)
   349  	state, ok := p.sinkManager.r.GetTableState(span)
   350  	require.True(t, ok)
   351  	require.Equal(t, tablepb.TableStatePreparing, state)
   352  
   353  	// Push the resolved ts, mock that sorterNode receive first resolved event.
   354  	p.sourceManager.r.Add(
   355  		span,
   356  		[]*model.PolymorphicEvent{{
   357  			CRTs: 101,
   358  			RawKV: &model.RawKVEntry{
   359  				OpType: model.OpTypeResolved,
   360  				CRTs:   101,
   361  			},
   362  		}}...,
   363  	)
   364  
   365  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   366  	require.Nil(t, err)
   367  	tester.MustApplyPatches()
   368  
   369  	done = p.IsAddTableSpanFinished(span, true)
   370  	require.True(t, done)
   371  	state, ok = p.sinkManager.r.GetTableState(span)
   372  	require.True(t, ok)
   373  	require.Equal(t, tablepb.TableStatePrepared, state)
   374  
   375  	// ignore duplicate add request
   376  	ok, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 30}, true)
   377  	require.NoError(t, err)
   378  	require.True(t, ok)
   379  	stats = p.sinkManager.r.GetTableStats(span)
   380  	require.Equal(t, model.Ts(20), stats.CheckpointTs)
   381  	require.Equal(t, model.Ts(20), stats.ResolvedTs)
   382  	require.Equal(t, model.Ts(20), stats.BarrierTs)
   383  
   384  	p.sinkManager.r.UpdateBarrierTs(50, nil)
   385  	stats = p.sinkManager.r.GetTableStats(span)
   386  	require.Equal(t, model.Ts(20), stats.ResolvedTs)
   387  	require.Equal(t, model.Ts(50), stats.BarrierTs)
   388  
   389  	// Start to replicate table-1.
   390  	ok, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 30, ResolvedTs: 60}, false)
   391  	require.NoError(t, err)
   392  	require.True(t, ok)
   393  
   394  	stats = p.sinkManager.r.GetTableStats(span)
   395  	require.Equal(t, model.Ts(60), stats.ResolvedTs)
   396  	require.Equal(t, model.Ts(50), stats.BarrierTs)
   397  
   398  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   399  	require.Nil(t, err)
   400  	tester.MustApplyPatches()
   401  
   402  	// table-1: `prepared` -> `replicating`
   403  	state, ok = p.sinkManager.r.GetTableState(span)
   404  	require.True(t, ok)
   405  	require.Equal(t, tablepb.TableStateReplicating, state)
   406  
   407  	err = p.Close()
   408  	require.Nil(t, err)
   409  	require.Nil(t, p.agent)
   410  }
   411  
   412  func TestProcessorError(t *testing.T) {
   413  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   414  	ctx := context.Background()
   415  	liveness := model.LivenessCaptureAlive
   416  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   417  
   418  	// init tick
   419  	require.Nil(t, p.lazyInit(ctx))
   420  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   421  	require.Nil(t, err)
   422  	createTaskPosition(changefeed, p.captureInfo)
   423  	tester.MustApplyPatches()
   424  
   425  	// send a abnormal error
   426  	p.sinkManager.errors <- cerror.ErrSinkURIInvalid
   427  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   428  	require.Error(t, err)
   429  	patchProcessorErr(p.captureInfo, changefeed, err)
   430  	tester.MustApplyPatches()
   431  	require.Equal(t, changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{
   432  		Error: &model.RunningError{
   433  			Time:    changefeed.TaskPositions[p.captureInfo.ID].Error.Time,
   434  			Addr:    "127.0.0.1:0000",
   435  			Code:    "CDC:ErrSinkURIInvalid",
   436  			Message: "[CDC:ErrSinkURIInvalid]sink uri invalid '%s'",
   437  		},
   438  	})
   439  	require.Nil(t, p.Close())
   440  	tester.MustApplyPatches()
   441  
   442  	p, tester, changefeed = initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   443  	// init tick
   444  	require.Nil(t, p.lazyInit(ctx))
   445  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   446  	require.Nil(t, err)
   447  	createTaskPosition(changefeed, p.captureInfo)
   448  	tester.MustApplyPatches()
   449  
   450  	// send a normal error
   451  	p.sinkManager.errors <- context.Canceled
   452  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   453  	patchProcessorErr(p.captureInfo, changefeed, err)
   454  	tester.MustApplyPatches()
   455  	require.True(t, cerror.ErrReactorFinished.Equal(errors.Cause(err)))
   456  	require.Equal(t, changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{
   457  		Error: nil,
   458  	})
   459  	require.Nil(t, p.Close())
   460  	tester.MustApplyPatches()
   461  }
   462  
   463  func TestProcessorExit(t *testing.T) {
   464  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   465  	liveness := model.LivenessCaptureAlive
   466  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   467  	// init tick
   468  	checkChangefeedNormal(changefeed)
   469  	require.Nil(t, p.lazyInit(context.Background()))
   470  	createTaskPosition(changefeed, p.captureInfo)
   471  	tester.MustApplyPatches()
   472  
   473  	// stop the changefeed
   474  	changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   475  		status.AdminJobType = model.AdminStop
   476  		return status, true, nil
   477  	})
   478  	tester.MustApplyPatches()
   479  	require.False(t, checkChangefeedNormal(changefeed))
   480  	tester.MustApplyPatches()
   481  	require.Equal(t, changefeed.TaskPositions[p.captureInfo.ID], &model.TaskPosition{
   482  		Error: nil,
   483  	})
   484  	require.Nil(t, p.Close())
   485  	tester.MustApplyPatches()
   486  }
   487  
   488  func TestProcessorClose(t *testing.T) {
   489  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   490  	ctx := context.Background()
   491  	liveness := model.LivenessCaptureAlive
   492  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   493  	// init tick
   494  	checkChangefeedNormal(changefeed)
   495  	require.Nil(t, p.lazyInit(ctx))
   496  	createTaskPosition(changefeed, p.captureInfo)
   497  	tester.MustApplyPatches()
   498  
   499  	// Do a no operation tick to lazy init the processor.
   500  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   501  	require.Nil(t, err)
   502  	tester.MustApplyPatches()
   503  
   504  	// add tables
   505  	done, err := p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 20}, false)
   506  	require.Nil(t, err)
   507  	require.True(t, done)
   508  	done, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(2), tablepb.Checkpoint{CheckpointTs: 30}, false)
   509  	require.Nil(t, err)
   510  	require.True(t, done)
   511  
   512  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   513  	require.Nil(t, err)
   514  	tester.MustApplyPatches()
   515  
   516  	// push the resolvedTs and checkpointTs
   517  	changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   518  		return status, true, nil
   519  	})
   520  	tester.MustApplyPatches()
   521  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   522  	require.Nil(t, err)
   523  	tester.MustApplyPatches()
   524  	require.Contains(t, changefeed.TaskPositions, p.captureInfo.ID)
   525  
   526  	require.Nil(t, p.Close())
   527  	tester.MustApplyPatches()
   528  	require.Nil(t, p.sinkManager.r)
   529  	require.Nil(t, p.sourceManager.r)
   530  	require.Nil(t, p.agent)
   531  
   532  	p, tester, changefeed = initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   533  	// init tick
   534  	checkChangefeedNormal(changefeed)
   535  	require.Nil(t, p.lazyInit(ctx))
   536  	createTaskPosition(changefeed, p.captureInfo)
   537  	tester.MustApplyPatches()
   538  
   539  	// Do a no operation tick to lazy init the processor.
   540  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   541  	require.Nil(t, err)
   542  	tester.MustApplyPatches()
   543  
   544  	// add tables
   545  	done, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 20}, false)
   546  	require.Nil(t, err)
   547  	require.True(t, done)
   548  	done, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(2), tablepb.Checkpoint{CheckpointTs: 30}, false)
   549  	require.Nil(t, err)
   550  	require.True(t, done)
   551  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   552  	require.Nil(t, err)
   553  	tester.MustApplyPatches()
   554  
   555  	// send error
   556  	p.sinkManager.errors <- cerror.ErrSinkURIInvalid
   557  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   558  	require.Error(t, err)
   559  	patchProcessorErr(p.captureInfo, changefeed, err)
   560  	tester.MustApplyPatches()
   561  
   562  	require.Nil(t, p.Close())
   563  	tester.MustApplyPatches()
   564  	require.Equal(t, changefeed.TaskPositions[p.captureInfo.ID].Error, &model.RunningError{
   565  		Time:    changefeed.TaskPositions[p.captureInfo.ID].Error.Time,
   566  		Addr:    "127.0.0.1:0000",
   567  		Code:    "CDC:ErrSinkURIInvalid",
   568  		Message: "[CDC:ErrSinkURIInvalid]sink uri invalid '%s'",
   569  	})
   570  	require.Nil(t, p.sinkManager.r)
   571  	require.Nil(t, p.sourceManager.r)
   572  	require.Nil(t, p.agent)
   573  }
   574  
   575  func TestPositionDeleted(t *testing.T) {
   576  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   577  	ctx := context.Background()
   578  	liveness := model.LivenessCaptureAlive
   579  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   580  	// init tick
   581  	checkChangefeedNormal(changefeed)
   582  	require.Nil(t, p.lazyInit(ctx))
   583  	createTaskPosition(changefeed, p.captureInfo)
   584  	tester.MustApplyPatches()
   585  	require.Contains(t, changefeed.TaskPositions, p.captureInfo.ID)
   586  
   587  	// Do a no operation tick to lazy init the processor.
   588  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   589  	require.Nil(t, err)
   590  	tester.MustApplyPatches()
   591  
   592  	// add table
   593  	done, err := p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(1), tablepb.Checkpoint{CheckpointTs: 30}, false)
   594  	require.Nil(t, err)
   595  	require.True(t, done)
   596  	done, err = p.AddTableSpan(ctx, spanz.TableIDToComparableSpan(2), tablepb.Checkpoint{CheckpointTs: 40}, false)
   597  	require.Nil(t, err)
   598  	require.True(t, done)
   599  
   600  	// some others delete the task position
   601  	changefeed.PatchTaskPosition(p.captureInfo.ID,
   602  		func(position *model.TaskPosition) (*model.TaskPosition, bool, error) {
   603  			return nil, true, nil
   604  		})
   605  	tester.MustApplyPatches()
   606  
   607  	// position created again
   608  	checkChangefeedNormal(changefeed)
   609  	createTaskPosition(changefeed, p.captureInfo)
   610  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   611  	require.Nil(t, err)
   612  	tester.MustApplyPatches()
   613  	require.Equal(t, &model.TaskPosition{}, changefeed.TaskPositions[p.captureInfo.ID])
   614  	require.Contains(t, changefeed.TaskPositions, p.captureInfo.ID)
   615  
   616  	require.Nil(t, p.Close())
   617  	tester.MustApplyPatches()
   618  }
   619  
   620  func TestSchemaGC(t *testing.T) {
   621  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   622  	ctx := context.Background()
   623  	liveness := model.LivenessCaptureAlive
   624  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   625  
   626  	var err error
   627  	// init tick
   628  	checkChangefeedNormal(changefeed)
   629  	require.Nil(t, p.lazyInit(ctx))
   630  	createTaskPosition(changefeed, p.captureInfo)
   631  	tester.MustApplyPatches()
   632  
   633  	updateChangeFeedPosition(t, tester,
   634  		model.DefaultChangeFeedID("changefeed-id-test"),
   635  		50)
   636  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   637  	require.Nil(t, err)
   638  	tester.MustApplyPatches()
   639  
   640  	// GC Ts should be (checkpoint - 1).
   641  	require.Equal(t, p.ddlHandler.r.schemaStorage.(*mockSchemaStorage).lastGcTs, uint64(49))
   642  	require.Equal(t, p.lastSchemaTs, uint64(49))
   643  
   644  	require.Nil(t, p.Close())
   645  	tester.MustApplyPatches()
   646  }
   647  
   648  //nolint:unused
   649  func updateChangeFeedPosition(t *testing.T, tester *orchestrator.ReactorStateTester, cfID model.ChangeFeedID, checkpointTs model.Ts) {
   650  	key := etcd.CDCKey{
   651  		ClusterID:    etcd.DefaultCDCClusterID,
   652  		Tp:           etcd.CDCKeyTypeChangeFeedStatus,
   653  		ChangefeedID: cfID,
   654  	}
   655  	keyStr := key.String()
   656  
   657  	cfStatus := &model.ChangeFeedStatus{
   658  		CheckpointTs: checkpointTs,
   659  	}
   660  	valueBytes, err := json.Marshal(cfStatus)
   661  	require.Nil(t, err)
   662  
   663  	tester.MustUpdate(keyStr, valueBytes)
   664  }
   665  
   666  func TestIgnorableError(t *testing.T) {
   667  	testCases := []struct {
   668  		err       error
   669  		ignorable bool
   670  	}{
   671  		{nil, true},
   672  		{cerror.ErrAdminStopProcessor.GenWithStackByArgs(), true},
   673  		{cerror.ErrReactorFinished.GenWithStackByArgs(), true},
   674  		{cerror.ErrRedoWriterStopped.GenWithStackByArgs(), false},
   675  		{errors.Trace(context.Canceled), true},
   676  		{cerror.ErrProcessorTableNotFound.GenWithStackByArgs(), false},
   677  		{errors.New("test error"), false},
   678  	}
   679  	for _, tc := range testCases {
   680  		require.Equal(t, isProcessorIgnorableError(tc.err), tc.ignorable)
   681  	}
   682  }
   683  
   684  func TestUpdateBarrierTs(t *testing.T) {
   685  	globalVars, changefeedInfo := vars.NewGlobalVarsAndChangefeedInfo4Test()
   686  	ctx := context.Background()
   687  	liveness := model.LivenessCaptureAlive
   688  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedInfo)
   689  	changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   690  		status.CheckpointTs = 5
   691  		return status, true, nil
   692  	})
   693  	p.ddlHandler.r.schemaStorage.(*mockSchemaStorage).resolvedTs = 10
   694  
   695  	// init tick
   696  	checkChangefeedNormal(changefeed)
   697  	require.Nil(t, p.lazyInit(ctx))
   698  	createTaskPosition(changefeed, p.captureInfo)
   699  	tester.MustApplyPatches()
   700  	require.Contains(t, changefeed.TaskPositions, p.captureInfo.ID)
   701  
   702  	// Do a no operation tick to lazy init the processor.
   703  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   704  	require.Nil(t, err)
   705  	tester.MustApplyPatches()
   706  
   707  	span := spanz.TableIDToComparableSpan(1)
   708  	done, err := p.AddTableSpan(ctx, span, tablepb.Checkpoint{CheckpointTs: 5}, false)
   709  	require.True(t, done)
   710  	require.Nil(t, err)
   711  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   712  	require.Nil(t, err)
   713  	tester.MustApplyPatches()
   714  
   715  	// Global resolved ts has advanced while schema storage stalls.
   716  	changefeed.PatchStatus(func(status *model.ChangeFeedStatus) (*model.ChangeFeedStatus, bool, error) {
   717  		return status, true, nil
   718  	})
   719  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   720  	require.Nil(t, err)
   721  	tester.MustApplyPatches()
   722  	p.updateBarrierTs(&schedulepb.Barrier{GlobalBarrierTs: 20, TableBarriers: nil})
   723  	status := p.sinkManager.r.GetTableStats(span)
   724  	require.Equal(t, uint64(10), status.BarrierTs)
   725  
   726  	// Schema storage has advanced too.
   727  	p.ddlHandler.r.schemaStorage.(*mockSchemaStorage).resolvedTs = 15
   728  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   729  	require.Nil(t, err)
   730  	tester.MustApplyPatches()
   731  	p.updateBarrierTs(&schedulepb.Barrier{GlobalBarrierTs: 20, TableBarriers: nil})
   732  	status = p.sinkManager.r.GetTableStats(span)
   733  	require.Equal(t, uint64(15), status.BarrierTs)
   734  
   735  	require.Nil(t, p.Close())
   736  	tester.MustApplyPatches()
   737  }
   738  
   739  func TestProcessorLiveness(t *testing.T) {
   740  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   741  	ctx := context.Background()
   742  	liveness := model.LivenessCaptureAlive
   743  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   744  
   745  	// First tick for creating position.
   746  	require.Nil(t, p.lazyInit(ctx))
   747  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   748  	require.Nil(t, err)
   749  	tester.MustApplyPatches()
   750  
   751  	// Second tick for init.
   752  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   753  	require.Nil(t, err)
   754  
   755  	// Changing p.liveness affects p.agent liveness.
   756  	p.liveness.Store(model.LivenessCaptureStopping)
   757  	require.Equal(t, model.LivenessCaptureStopping, p.agent.(*mockAgent).liveness.Load())
   758  
   759  	// Changing p.agent liveness affects p.liveness.
   760  	// Force set liveness to alive.
   761  	*p.agent.(*mockAgent).liveness = model.LivenessCaptureAlive
   762  	require.Equal(t, model.LivenessCaptureAlive, p.liveness.Load())
   763  
   764  	require.Nil(t, p.Close())
   765  	tester.MustApplyPatches()
   766  }
   767  
   768  func TestProcessorDostNotStuckInInit(t *testing.T) {
   769  	_ = failpoint.
   770  		Enable("github.com/pingcap/tiflow/cdc/processor/sinkmanager/SinkManagerRunError",
   771  			"1*return(true)")
   772  	defer func() {
   773  		_ = failpoint.
   774  			Disable("github.com/pingcap/tiflow/cdc/processor/sinkmanager/SinkManagerRunError")
   775  	}()
   776  
   777  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   778  	ctx := context.Background()
   779  	liveness := model.LivenessCaptureAlive
   780  	p, tester, changefeed := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   781  	require.Nil(t, p.lazyInit(ctx))
   782  
   783  	// First tick for creating position.
   784  	err, _ := p.Tick(ctx, changefeed.Info, changefeed.Status)
   785  	require.Nil(t, err)
   786  	tester.MustApplyPatches()
   787  
   788  	// Second tick for init.
   789  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   790  	require.Nil(t, err)
   791  
   792  	// TODO(qupeng): third tick for handle a warning.
   793  	err, _ = p.Tick(ctx, changefeed.Info, changefeed.Status)
   794  	require.Nil(t, err)
   795  
   796  	require.Nil(t, p.Close())
   797  	tester.MustApplyPatches()
   798  }
   799  
   800  func TestProcessorNotInitialized(t *testing.T) {
   801  	globalVars, changefeedVars := vars.NewGlobalVarsAndChangefeedInfo4Test()
   802  	liveness := model.LivenessCaptureAlive
   803  	p, _, _ := initProcessor4Test(t, &liveness, false, globalVars, changefeedVars)
   804  	require.Nil(t, p.WriteDebugInfo(os.Stdout))
   805  }