From 2b270b22e47d362b880d332f6234310a657a8546 Mon Sep 17 00:00:00 2001 From: zhaoxinyu Date: Fri, 13 May 2022 14:20:35 +0800 Subject: [PATCH 1/3] owner/(ticdc): fix the disorder problem for ddl events execution (#5408) close pingcap/tiflow#5406 --- cdc/model/sink.go | 1 + cdc/owner/changefeed.go | 15 +++------------ cdc/owner/changefeed_test.go | 34 +++++++++++++++++----------------- cdc/owner/ddl_sink.go | 21 +++++++-------------- 4 files changed, 28 insertions(+), 43 deletions(-) diff --git a/cdc/model/sink.go b/cdc/model/sink.go index 8c0ca50cecf..e68e4453cce 100644 --- a/cdc/model/sink.go +++ b/cdc/model/sink.go @@ -468,6 +468,7 @@ type DDLEvent struct { PreTableInfo *SimpleTableInfo `msg:"pre-table-info"` Query string `msg:"query"` Type model.ActionType `msg:"-"` + Done bool `msg:"-"` } // RedoDDLEvent represents DDL event used in redo log persistent diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 22236d5e86d..619510d9c1f 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -61,7 +61,7 @@ type changefeed struct { // a DDL job asynchronously. After the DDL job has been executed, // ddlEventCache will be set to nil. ddlEventCache contains more than // one event for a rename tables DDL job. - ddlEventCache map[*model.DDLEvent]bool + ddlEventCache []*model.DDLEvent // currentTableNames is the table names that the changefeed is watching. // And it contains only the tables of the ddl that have been processed. // The ones that have not been executed yet do not have. @@ -569,10 +569,7 @@ func (c *changefeed) asyncExecDDLJob(ctx cdcContext.Context, if err != nil { return false, errors.Trace(err) } - c.ddlEventCache = make(map[*model.DDLEvent]bool) - for _, event := range ddlEvents { - c.ddlEventCache[event] = false - } + c.ddlEventCache = ddlEvents for _, ddlEvent := range ddlEvents { if c.redoManager.Enabled() { err = c.redoManager.EmitDDLEvent(ctx, ddlEvent) @@ -584,17 +581,11 @@ func (c *changefeed) asyncExecDDLJob(ctx cdcContext.Context, } jobDone := true - for event, done := range c.ddlEventCache { - if done { - continue - } + for _, event := range c.ddlEventCache { eventDone, err := c.asyncExecDDLEvent(ctx, event) if err != nil { return false, err } - if eventDone { - c.ddlEventCache[event] = true - } jobDone = jobDone && eventDone } diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 9d6632ac06c..3a57a31d02b 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -805,12 +805,12 @@ func TestExecRenameTablesDDL(t *testing.T) { oldTableIDs = append(oldTableIDs, job.TableID) } require.Nil(t, err) - require.Equal(t, done, false) + require.Equal(t, false, done) require.Equal(t, expectedDDL, mockDDLSink.ddlExecuting.Query) mockDDLSink.ddlDone = true done, err = cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, true) + require.Equal(t, true, done) require.Equal(t, expectedDDL, mockDDLSink.ddlExecuting.Query) } @@ -852,19 +852,19 @@ func TestExecRenameTablesDDL(t *testing.T) { mockDDLSink.recordDDLHistory = true done, err := cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, false) + require.Equal(t, false, done) require.Len(t, mockDDLSink.ddlHistory, 2) - require.Equal(t, mockDDLSink.ddlHistory[0], - "RENAME TABLE `test1`.`tb1` TO `test2`.`tb10`") - require.Equal(t, mockDDLSink.ddlHistory[1], - "RENAME TABLE `test2`.`tb2` TO `test1`.`tb20`") + require.Equal(t, "RENAME TABLE `test1`.`tb1` TO `test2`.`tb10`", + mockDDLSink.ddlHistory[0]) + require.Equal(t, "RENAME TABLE `test2`.`tb2` TO `test1`.`tb20`", + mockDDLSink.ddlHistory[1]) // mock all of the rename table statements have been done mockDDLSink.resetDDLDone = false mockDDLSink.ddlDone = true done, err = cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, true) + require.Equal(t, true, done) } func TestExecDropTablesDDL(t *testing.T) { @@ -886,12 +886,12 @@ func TestExecDropTablesDDL(t *testing.T) { job := helper.DDL2Job(actualDDL) done, err := cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, false) + require.Equal(t, false, done) require.Equal(t, expectedDDL, mockDDLSink.ddlExecuting.Query) mockDDLSink.ddlDone = true done, err = cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, true) + require.Equal(t, true, done) } execCreateStmt("create database test1", @@ -909,12 +909,12 @@ func TestExecDropTablesDDL(t *testing.T) { execDropStmt := func(job *timodel.Job, expectedDDL string) { done, err := cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, false) + require.Equal(t, false, done) require.Equal(t, mockDDLSink.ddlExecuting.Query, expectedDDL) mockDDLSink.ddlDone = true done, err = cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, true) + require.Equal(t, true, done) } execDropStmt(jobs[0], "DROP TABLE `test1`.`tb2`") @@ -940,12 +940,12 @@ func TestExecDropViewsDDL(t *testing.T) { job := helper.DDL2Job(actualDDL) done, err := cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, false) + require.Equal(t, false, done) require.Equal(t, expectedDDL, mockDDLSink.ddlExecuting.Query) mockDDLSink.ddlDone = true done, err = cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, true) + require.Equal(t, true, done) } execCreateStmt("create database test1", "CREATE DATABASE `test1`") @@ -970,12 +970,12 @@ func TestExecDropViewsDDL(t *testing.T) { execDropStmt := func(job *timodel.Job, expectedDDL string) { done, err := cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, false) - require.Equal(t, mockDDLSink.ddlExecuting.Query, expectedDDL) + require.Equal(t, false, done) + require.Equal(t, expectedDDL, mockDDLSink.ddlExecuting.Query) mockDDLSink.ddlDone = true done, err = cf.asyncExecDDLJob(ctx, job) require.Nil(t, err) - require.Equal(t, done, true) + require.Equal(t, true, done) } execDropStmt(jobs[0], "DROP VIEW `test1`.`view2`") diff --git a/cdc/owner/ddl_sink.go b/cdc/owner/ddl_sink.go index 1e033b67442..96f51d56b17 100644 --- a/cdc/owner/ddl_sink.go +++ b/cdc/owner/ddl_sink.go @@ -65,9 +65,6 @@ type ddlSinkImpl struct { checkpointTs model.Ts currentTableNames []model.TableName } - // ddlFinishedTsMap is used to check whether a ddl event in a ddl job has - // been executed successfully. - ddlFinishedTsMap sync.Map // ddlSentTsMap is used to check whether a ddl event in a ddl job has been // sent to `ddlCh` successfully. ddlSentTsMap map[*model.DDLEvent]model.Ts @@ -198,10 +195,10 @@ func (s *ddlSinkImpl) run(ctx cdcContext.Context, id model.ChangeFeedID, info *m zap.String("changefeed", ctx.ChangefeedVars().ID.ID), zap.Bool("ignored", err != nil), zap.Any("ddl", ddl)) - s.ddlFinishedTsMap.Store(ddl, ddl.CommitTs) // Force emitting checkpoint ts when a ddl event is finished. // Otherwise, a kafka consumer may not execute that ddl event. s.mu.Lock() + ddl.Done = true checkpointTs := s.mu.checkpointTs if checkpointTs == 0 || checkpointTs <= lastCheckpointTs { s.mu.Unlock() @@ -242,22 +239,18 @@ func (s *ddlSinkImpl) emitCheckpointTs(ts uint64, tableNames []model.TableName) // and CommitTs. So in emitDDLEvent, we get the DDL finished ts of an event // from a map in order to check whether that event is finshed or not. func (s *ddlSinkImpl) emitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) (bool, error) { - var ddlFinishedTs model.Ts - - ts, ok := s.ddlFinishedTsMap.Load(ddl) - if ok { - ddlFinishedTs = ts.(model.Ts) - } - if ddl.CommitTs <= ddlFinishedTs { + s.mu.Lock() + if ddl.Done { // the DDL event is executed successfully, and done is true log.Info("ddl already executed", zap.String("namespace", ctx.ChangefeedVars().ID.Namespace), zap.String("changefeed", ctx.ChangefeedVars().ID.ID), - zap.Uint64("ddlFinishedTs", ddlFinishedTs), zap.Any("DDL", ddl)) - s.ddlFinishedTsMap.Delete(ddl) + zap.Any("DDL", ddl)) delete(s.ddlSentTsMap, ddl) + s.mu.Unlock() return true, nil } + s.mu.Unlock() ddlSentTs := s.ddlSentTsMap[ddl] if ddl.CommitTs <= ddlSentTs { @@ -282,7 +275,7 @@ func (s *ddlSinkImpl) emitDDLEvent(ctx cdcContext.Context, ddl *model.DDLEvent) zap.String("namespace", ctx.ChangefeedVars().ID.Namespace), zap.String("changefeed", ctx.ChangefeedVars().ID.ID), zap.Uint64("ddlSentTs", ddlSentTs), - zap.Uint64("ddlFinishedTs", ddlFinishedTs), zap.Any("DDL", ddl)) + zap.Any("DDL", ddl)) // if this hit, we think that ddlCh is full, // just return false and send the ddl in the next round. } From a4784964de592e889cdd7fff6fa4fc46df123329 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Fri, 13 May 2022 15:34:36 +0800 Subject: [PATCH 2/3] cdc: move scheduler and agent to package base (#5396) close pingcap/tiflow#5395 --- cdc/owner/changefeed.go | 36 ++- cdc/owner/changefeed_test.go | 11 +- cdc/processor/processor.go | 54 +++-- cdc/processor/processor_test.go | 4 +- cdc/scheduler/internal/agent.go | 36 +++ cdc/scheduler/{ => internal/base}/agent.go | 119 ++++------ .../{ => internal/base}/agent_mock.go | 36 ++- .../{ => internal/base}/agent_test.go | 35 +-- cdc/scheduler/{ => internal/base}/balancer.go | 4 +- .../{ => internal/base}/balancer_test.go | 4 +- .../{ => internal/base}/checkpoint_sender.go | 4 +- .../base}/checkpoint_sender_test.go | 2 +- .../{ => internal/base}/info_provider.go | 30 +-- .../{ => internal/base}/info_provider_test.go | 9 +- .../{ => internal/base}/move_table_manager.go | 4 +- .../base}/move_table_manager_test.go | 10 +- .../internal/base/owner_scheduler.go} | 135 +++++------- .../internal/base/owner_scheduler_test.go} | 119 ++++------ .../internal/base/processor_agent.go} | 99 ++++----- .../internal/base/processor_agent_test.go} | 110 +++++----- .../internal/base/protocol}/protocol.go | 29 +-- .../internal/base/protocol}/protocol_test.go | 13 +- .../base}/schedule_dispatcher.go | 143 ++++++------ .../base}/schedule_dispatcher_test.go | 206 ++++++++++-------- cdc/scheduler/internal/info_provider.go | 36 +++ cdc/scheduler/internal/scheduler.go | 58 +++++ cdc/scheduler/internal/table_executor.go | 46 ++++ .../{ => internal}/util/sort_table_ids.go | 0 .../{ => internal}/util/table_set.go | 0 .../{ => internal}/util/table_set_test.go | 0 cdc/scheduler/rexport.go | 79 +++++++ 31 files changed, 831 insertions(+), 640 deletions(-) create mode 100644 cdc/scheduler/internal/agent.go rename cdc/scheduler/{ => internal/base}/agent.go (78%) rename cdc/scheduler/{ => internal/base}/agent_mock.go (79%) rename cdc/scheduler/{ => internal/base}/agent_test.go (93%) rename cdc/scheduler/{ => internal/base}/balancer.go (98%) rename cdc/scheduler/{ => internal/base}/balancer_test.go (98%) rename cdc/scheduler/{ => internal/base}/checkpoint_sender.go (98%) rename cdc/scheduler/{ => internal/base}/checkpoint_sender_test.go (99%) rename cdc/scheduler/{ => internal/base}/info_provider.go (73%) rename cdc/scheduler/{ => internal/base}/info_provider_test.go (92%) rename cdc/scheduler/{ => internal/base}/move_table_manager.go (99%) rename cdc/scheduler/{ => internal/base}/move_table_manager_test.go (94%) rename cdc/{owner/scheduler.go => scheduler/internal/base/owner_scheduler.go} (66%) rename cdc/{owner/scheduler_test.go => scheduler/internal/base/owner_scheduler_test.go} (63%) rename cdc/{processor/agent.go => scheduler/internal/base/processor_agent.go} (84%) rename cdc/{processor/agent_test.go => scheduler/internal/base/processor_agent_test.go} (83%) rename cdc/{model => scheduler/internal/base/protocol}/protocol.go (83%) rename cdc/{model => scheduler/internal/base/protocol}/protocol_test.go (91%) rename cdc/scheduler/{ => internal/base}/schedule_dispatcher.go (80%) rename cdc/scheduler/{ => internal/base}/schedule_dispatcher_test.go (84%) create mode 100644 cdc/scheduler/internal/info_provider.go create mode 100644 cdc/scheduler/internal/scheduler.go create mode 100644 cdc/scheduler/internal/table_executor.go rename cdc/scheduler/{ => internal}/util/sort_table_ids.go (100%) rename cdc/scheduler/{ => internal}/util/table_set.go (100%) rename cdc/scheduler/{ => internal}/util/table_set_test.go (100%) create mode 100644 cdc/scheduler/rexport.go diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 619510d9c1f..9d4028149fa 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/redo" - schedulerv2 "github.com/pingcap/tiflow/cdc/scheduler" + "github.com/pingcap/tiflow/cdc/scheduler" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" @@ -39,12 +39,33 @@ import ( "go.uber.org/zap" ) +// newSchedulerV2FromCtx creates a new schedulerV2 from context. +// This function is factored out to facilitate unit testing. +func newSchedulerV2FromCtx( + ctx cdcContext.Context, startTs uint64, +) (scheduler.Scheduler, error) { + changeFeedID := ctx.ChangefeedVars().ID + messageServer := ctx.GlobalVars().MessageServer + messageRouter := ctx.GlobalVars().MessageRouter + ownerRev := ctx.GlobalVars().OwnerRevision + ret, err := scheduler.NewScheduler( + ctx, changeFeedID, startTs, messageServer, messageRouter, ownerRev) + if err != nil { + return nil, errors.Trace(err) + } + return ret, nil +} + +func newScheduler(ctx cdcContext.Context, startTs uint64) (scheduler.Scheduler, error) { + return newSchedulerV2FromCtx(ctx, startTs) +} + type changefeed struct { id model.ChangeFeedID state *orchestrator.ChangefeedReactorState upStream *upstream.Upstream - scheduler scheduler + scheduler scheduler.Scheduler barriers *barriers feedStateManager *feedStateManager redoManager redo.LogManager @@ -85,7 +106,7 @@ type changefeed struct { newDDLPuller func(ctx cdcContext.Context, upStream *upstream.Upstream, startTs uint64) (DDLPuller, error) newSink func() DDLSink - newScheduler func(ctx cdcContext.Context, startTs uint64) (scheduler, error) + newScheduler func(ctx cdcContext.Context, startTs uint64) (scheduler.Scheduler, error) } func newChangefeed(id model.ChangeFeedID, upStream *upstream.Upstream) *changefeed { @@ -233,7 +254,8 @@ func (c *changefeed) tick(ctx cdcContext.Context, state *orchestrator.Changefeed } startTime := time.Now() - newCheckpointTs, newResolvedTs, err := c.scheduler.Tick(ctx, c.state, c.schema.AllPhysicalTables(), captures) + newCheckpointTs, newResolvedTs, err := c.scheduler.Tick( + ctx, c.state.Status.CheckpointTs, c.schema.AllPhysicalTables(), captures) costTime := time.Since(startTime) if costTime > schedulerLogsWarnDuration { log.Warn("scheduler tick took too long", @@ -249,7 +271,7 @@ func (c *changefeed) tick(ctx cdcContext.Context, state *orchestrator.Changefeed // CheckpointCannotProceed implies that not all tables are being replicated normally, // so in that case there is no need to advance the global watermarks. - if newCheckpointTs != schedulerv2.CheckpointCannotProceed { + if newCheckpointTs != scheduler.CheckpointCannotProceed { if newResolvedTs > barrierTs { newResolvedTs = barrierTs } @@ -660,8 +682,8 @@ func (c *changefeed) Close(ctx cdcContext.Context) { } // GetInfoProvider returns an InfoProvider if one is available. -func (c *changefeed) GetInfoProvider() schedulerv2.InfoProvider { - if provider, ok := c.scheduler.(schedulerv2.InfoProvider); ok { +func (c *changefeed) GetInfoProvider() scheduler.InfoProvider { + if provider, ok := c.scheduler.(scheduler.InfoProvider); ok { return provider } return nil diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 3a57a31d02b..06cffdf3f5c 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -28,6 +28,7 @@ import ( timodel "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tiflow/cdc/entry" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/scheduler" "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/orchestrator" @@ -147,8 +148,8 @@ type mockScheduler struct { } func (m *mockScheduler) Tick( - ctx cdcContext.Context, - state *orchestrator.ChangefeedReactorState, + ctx context.Context, + checkpointTs model.Ts, currentTables []model.TableID, captures map[model.CaptureID]*model.CaptureInfo, ) (newCheckpointTs, newResolvedTs model.Ts, err error) { @@ -163,7 +164,7 @@ func (m *mockScheduler) MoveTable(tableID model.TableID, target model.CaptureID) func (m *mockScheduler) Rebalance() {} // Close closes the scheduler and releases resources. -func (m *mockScheduler) Close(ctx cdcContext.Context) {} +func (m *mockScheduler) Close(ctx context.Context) {} func createChangefeed4Test(ctx cdcContext.Context, t *testing.T) ( *changefeed, *orchestrator.ChangefeedReactorState, @@ -183,7 +184,9 @@ func createChangefeed4Test(ctx cdcContext.Context, t *testing.T) ( recordDDLHistory: false, } }) - cf.newScheduler = func(ctx cdcContext.Context, startTs uint64) (scheduler, error) { + cf.newScheduler = func( + ctx cdcContext.Context, startTs uint64, + ) (scheduler.Scheduler, error) { return &mockScheduler{}, nil } cf.upStream = upStream diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index b5a027638e2..0a3bafa337c 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -32,6 +32,7 @@ import ( tablepipeline "github.com/pingcap/tiflow/cdc/processor/pipeline" "github.com/pingcap/tiflow/cdc/puller" "github.com/pingcap/tiflow/cdc/redo" + "github.com/pingcap/tiflow/cdc/scheduler" "github.com/pingcap/tiflow/cdc/sink" "github.com/pingcap/tiflow/cdc/sink/metrics" "github.com/pingcap/tiflow/cdc/sorter/memory" @@ -80,9 +81,9 @@ type processor struct { lazyInit func(ctx cdcContext.Context) error createTablePipeline func(ctx cdcContext.Context, tableID model.TableID, replicaInfo *model.TableReplicaInfo) (tablepipeline.TablePipeline, error) - newAgent func(ctx cdcContext.Context) (processorAgent, error) + newAgent func(ctx cdcContext.Context) (scheduler.Agent, error) - agent processorAgent + agent scheduler.Agent checkpointTs model.Ts resolvedTs model.Ts @@ -104,15 +105,19 @@ func (p *processor) checkReadyForMessages() bool { } // AddTable implements TableExecutor interface. -func (p *processor) AddTable(ctx cdcContext.Context, tableID model.TableID, startTs model.Ts) (bool, error) { +func (p *processor) AddTable( + ctx context.Context, tableID model.TableID, startTs model.Ts, +) (bool, error) { if !p.checkReadyForMessages() { return false, nil } log.Info("adding table", zap.Int64("tableID", tableID), - cdcContext.ZapFieldChangefeed(ctx)) - err := p.addTable(ctx, tableID, &model.TableReplicaInfo{StartTs: startTs}) + zap.Uint64("checkpointTs", startTs), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID)) + err := p.addTable(ctx.(cdcContext.Context), tableID, &model.TableReplicaInfo{StartTs: startTs}) if err != nil { return false, errors.Trace(err) } @@ -120,7 +125,7 @@ func (p *processor) AddTable(ctx cdcContext.Context, tableID model.TableID, star } // RemoveTable implements TableExecutor interface. -func (p *processor) RemoveTable(ctx cdcContext.Context, tableID model.TableID) (bool, error) { +func (p *processor) RemoveTable(ctx context.Context, tableID model.TableID) (bool, error) { if !p.checkReadyForMessages() { return false, nil } @@ -128,7 +133,9 @@ func (p *processor) RemoveTable(ctx cdcContext.Context, tableID model.TableID) ( table, ok := p.tables[tableID] if !ok { log.Warn("table which will be deleted is not found", - cdcContext.ZapFieldChangefeed(ctx), zap.Int64("tableID", tableID)) + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), + zap.Int64("tableID", tableID)) return true, nil } @@ -137,7 +144,8 @@ func (p *processor) RemoveTable(ctx cdcContext.Context, tableID model.TableID) ( // We use a Debug log because it is conceivable for the pipeline to block for a legitimate reason, // and we do not want to alarm the user. log.Debug("AsyncStop has failed, possible due to a full pipeline", - cdcContext.ZapFieldChangefeed(ctx), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), zap.Uint64("checkpointTs", table.CheckpointTs()), zap.Int64("tableID", tableID)) return false, nil @@ -146,7 +154,7 @@ func (p *processor) RemoveTable(ctx cdcContext.Context, tableID model.TableID) ( } // IsAddTableFinished implements TableExecutor interface. -func (p *processor) IsAddTableFinished(ctx cdcContext.Context, tableID model.TableID) bool { +func (p *processor) IsAddTableFinished(ctx context.Context, tableID model.TableID) bool { if !p.checkReadyForMessages() { return false } @@ -154,7 +162,8 @@ func (p *processor) IsAddTableFinished(ctx cdcContext.Context, tableID model.Tab table, exist := p.tables[tableID] if !exist { log.Panic("table which was added is not found", - cdcContext.ZapFieldChangefeed(ctx), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), zap.Int64("tableID", tableID)) } localResolvedTs := p.resolvedTs @@ -176,13 +185,14 @@ func (p *processor) IsAddTableFinished(ctx cdcContext.Context, tableID model.Tab return false } log.Info("Add Table finished", - cdcContext.ZapFieldChangefeed(ctx), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), zap.Int64("tableID", tableID)) return true } // IsRemoveTableFinished implements TableExecutor interface. -func (p *processor) IsRemoveTableFinished(ctx cdcContext.Context, tableID model.TableID) bool { +func (p *processor) IsRemoveTableFinished(ctx context.Context, tableID model.TableID) bool { if !p.checkReadyForMessages() { return false } @@ -190,13 +200,15 @@ func (p *processor) IsRemoveTableFinished(ctx cdcContext.Context, tableID model. table, exist := p.tables[tableID] if !exist { log.Panic("table which was deleted is not found", - cdcContext.ZapFieldChangefeed(ctx), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), zap.Int64("tableID", tableID)) return true } if table.Status() != tablepipeline.TableStatusStopped { log.Debug("the table is still not stopped", - cdcContext.ZapFieldChangefeed(ctx), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), zap.Uint64("checkpointTs", table.CheckpointTs()), zap.Int64("tableID", tableID)) return false @@ -206,7 +218,8 @@ func (p *processor) IsRemoveTableFinished(ctx cdcContext.Context, tableID model. table.Wait() delete(p.tables, tableID) log.Info("Remove Table finished", - cdcContext.ZapFieldChangefeed(ctx), + zap.String("namespace", p.changefeedID.Namespace), + zap.String("changefeed", p.changefeedID.ID), zap.Int64("tableID", tableID)) return true @@ -475,14 +488,14 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { } opts[metrics.OptCaptureAddr] = ctx.GlobalVars().CaptureInfo.AdvertiseAddr log.Info("processor try new sink", - zap.String("namespace", p.changefeed.ID.Namespace), + zap.String("namespace", p.changefeedID.Namespace), zap.String("changefeed", p.changefeed.ID.ID)) start := time.Now() s, err := sink.New(stdCtx, p.changefeed.ID, p.changefeed.Info.SinkURI, p.filter, p.changefeed.Info.Config, opts, errCh) if err != nil { log.Info("processor new sink failed", - zap.String("namespace", p.changefeed.ID.Namespace), + zap.String("namespace", p.changefeedID.Namespace), zap.String("changefeed", p.changefeed.ID.ID), zap.Duration("duration", time.Since(start))) return errors.Trace(err) @@ -509,10 +522,11 @@ func (p *processor) lazyInitImpl(ctx cdcContext.Context) error { return nil } -func (p *processor) newAgentImpl(ctx cdcContext.Context) (processorAgent, error) { +func (p *processor) newAgentImpl(ctx cdcContext.Context) (scheduler.Agent, error) { messageServer := ctx.GlobalVars().MessageServer messageRouter := ctx.GlobalVars().MessageRouter - ret, err := newAgent(ctx, messageServer, messageRouter, p, p.changefeedID) + etcdClient := ctx.GlobalVars().EtcdClient + ret, err := scheduler.NewAgent(ctx, messageServer, messageRouter, etcdClient, p, p.changefeedID) if err != nil { return nil, errors.Trace(err) } @@ -882,7 +896,7 @@ func (p *processor) flushRedoLogMeta(ctx context.Context) error { func (p *processor) Close() error { log.Info("processor closing ...", - zap.String("namespace", p.changefeed.ID.Namespace), + zap.String("namespace", p.changefeedID.Namespace), zap.String("changefeed", p.changefeed.ID.ID)) for _, tbl := range p.tables { tbl.Cancel() diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index efe500408b9..804021908f0 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -204,14 +204,14 @@ func (s *mockSchemaStorage) DoGC(ts uint64) uint64 { type mockAgent struct { // dummy to satisfy the interface - processorAgent + scheduler.Agent executor scheduler.TableExecutor lastCheckpointTs model.Ts isClosed bool } -func (a *mockAgent) Tick(_ cdcContext.Context) error { +func (a *mockAgent) Tick(_ context.Context) error { if len(a.executor.GetAllCurrentTables()) == 0 { return nil } diff --git a/cdc/scheduler/internal/agent.go b/cdc/scheduler/internal/agent.go new file mode 100644 index 00000000000..3afe121abf4 --- /dev/null +++ b/cdc/scheduler/internal/agent.go @@ -0,0 +1,36 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "context" + + "github.com/pingcap/tiflow/cdc/model" +) + +// Agent is an interface for an object inside Processor that is responsible +// for receiving commands from the Owner. +// Ideally the processor should drive the Agent by Tick. +// +// Note that Agent is not thread-safe +type Agent interface { + // Tick is called periodically by the processor to drive the Agent's internal logic. + Tick(ctx context.Context) error + + // GetLastSentCheckpointTs returns the last checkpoint-ts already sent to the Owner. + GetLastSentCheckpointTs() (checkpointTs model.Ts) + + // Close closes the messenger and does the necessary cleanup. + Close() error +} diff --git a/cdc/scheduler/agent.go b/cdc/scheduler/internal/base/agent.go similarity index 78% rename from cdc/scheduler/agent.go rename to cdc/scheduler/internal/base/agent.go index 50bf26a9bb6..85edfa022a0 100644 --- a/cdc/scheduler/agent.go +++ b/cdc/scheduler/internal/base/agent.go @@ -11,9 +11,10 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "sync" "time" @@ -22,86 +23,58 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" - "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/uber-go/atomic" "go.uber.org/zap" ) -// Agent is an interface for an object inside Processor that is responsible -// for receiving commands from the Owner. -// Ideally the processor should drive the Agent by Tick. -type Agent interface { - // Tick is called periodically by the processor to drive the Agent's internal logic. - Tick(ctx context.Context) error - - // GetLastSentCheckpointTs returns the last checkpoint-ts already sent to the Owner. - GetLastSentCheckpointTs() (checkpointTs model.Ts) -} - -// TableExecutor is an abstraction for "Processor". -// -// This interface is so designed that it would be the least problematic -// to adapt the current Processor implementation to it. -// TODO find a way to make the semantics easier to understand. -type TableExecutor interface { - AddTable(ctx context.Context, tableID model.TableID, startTs model.Ts) (done bool, err error) - RemoveTable(ctx context.Context, tableID model.TableID) (done bool, err error) - IsAddTableFinished(ctx context.Context, tableID model.TableID) (done bool) - IsRemoveTableFinished(ctx context.Context, tableID model.TableID) (done bool) - - // GetAllCurrentTables should return all tables that are being run, - // being added and being removed. - // - // NOTE: two subsequent calls to the method should return the same - // result, unless there is a call to AddTable, RemoveTable, IsAddTableFinished - // or IsRemoveTableFinished in between two calls to this method. - GetAllCurrentTables() []model.TableID - - // GetCheckpoint returns the local checkpoint-ts and resolved-ts of - // the processor. Its calculation should take into consideration all - // tables that would have been returned if GetAllCurrentTables had been - // called immediately before. - GetCheckpoint() (checkpointTs, resolvedTs model.Ts) -} - // ProcessorMessenger implements how messages should be sent to the owner, // and should be able to know whether there are any messages not yet acknowledged // by the owner. type ProcessorMessenger interface { // FinishTableOperation notifies the owner that a table operation has finished. - FinishTableOperation(ctx context.Context, tableID model.TableID, epoch model.ProcessorEpoch) (done bool, err error) + FinishTableOperation( + ctx context.Context, tableID model.TableID, epoch protocol.ProcessorEpoch, + ) (done bool, err error) // SyncTaskStatuses informs the owner of the processor's current internal state. - SyncTaskStatuses(ctx context.Context, epoch model.ProcessorEpoch, adding, removing, running []model.TableID) (done bool, err error) - // SendCheckpoint sends the owner the processor's local watermarks, i.e., checkpoint-ts and resolved-ts. - SendCheckpoint(ctx context.Context, checkpointTs model.Ts, resolvedTs model.Ts) (done bool, err error) + SyncTaskStatuses( + ctx context.Context, epoch protocol.ProcessorEpoch, + adding, removing, running []model.TableID, + ) (done bool, err error) + // SendCheckpoint sends the owner the processor's local watermarks, + // i.e., checkpoint-ts and resolved-ts. + SendCheckpoint( + ctx context.Context, checkpointTs model.Ts, resolvedTs model.Ts, + ) (done bool, err error) // Barrier returns whether there is a pending message not yet acknowledged by the owner. Barrier(ctx context.Context) (done bool) // OnOwnerChanged is called when the owner is changed. - OnOwnerChanged(ctx context.Context, - newOwnerCaptureID model.CaptureID, - newOwnerRevision int64) + OnOwnerChanged( + ctx context.Context, newOwnerCaptureID model.CaptureID, newOwnerRevision int64, + ) // Close closes the messenger and does the necessary cleanup. Close() error } -// BaseAgentConfig stores configurations for BaseAgent -type BaseAgentConfig struct { +// AgentConfig stores configurations for BaseAgent +type AgentConfig struct { // SendCheckpointTsInterval is the interval to send checkpoint-ts to the owner. SendCheckpointTsInterval time.Duration } -// BaseAgent is an implementation of Agent. +// Agent is an implementation of Agent. // It implements the basic logic and is useful only if the Processor // implements its own TableExecutor and ProcessorMessenger. -type BaseAgent struct { - executor TableExecutor +type Agent struct { + executor internal.TableExecutor communicator ProcessorMessenger epochMu sync.RWMutex // epoch is reset on each Sync message. - epoch model.ProcessorEpoch + epoch protocol.ProcessorEpoch // pendingOpsMu protects pendingOps. // Note that we need a mutex because some methods are expected @@ -130,21 +103,21 @@ type BaseAgent struct { ownerHasChanged *atomic.Bool // read-only fields - config *BaseAgentConfig + config *AgentConfig logger *zap.Logger } // NewBaseAgent creates a new BaseAgent. func NewBaseAgent( changeFeedID model.ChangeFeedID, - executor TableExecutor, + executor internal.TableExecutor, messenger ProcessorMessenger, - config *BaseAgentConfig, -) *BaseAgent { + config *AgentConfig, +) *Agent { logger := log.L().With( zap.String("namespace", changeFeedID.Namespace), zap.String("changefeed", changeFeedID.ID)) - ret := &BaseAgent{ + ret := &Agent{ pendingOps: deque.NewDeque(), tableOperations: map[model.TableID]*agentOperation{}, logger: logger, @@ -172,7 +145,7 @@ type agentOperation struct { TableID model.TableID StartTs model.Ts IsDelete bool - Epoch model.ProcessorEpoch + Epoch protocol.ProcessorEpoch // FromOwnerID is for debugging purposesFromOwnerID FromOwnerID model.CaptureID @@ -191,7 +164,7 @@ type ownerInfo struct { } // Tick implements the interface Agent. -func (a *BaseAgent) Tick(ctx context.Context) error { +func (a *Agent) Tick(ctx context.Context) error { if a.ownerHasChanged.Swap(false) { // We need to notify the communicator if the owner has changed. // This is necessary because the communicator might be waiting for @@ -242,11 +215,11 @@ func (a *BaseAgent) Tick(ctx context.Context) error { } // GetLastSentCheckpointTs implements the interface Agent. -func (a *BaseAgent) GetLastSentCheckpointTs() model.Ts { +func (a *Agent) GetLastSentCheckpointTs() model.Ts { return a.checkpointSender.LastSentCheckpointTs() } -func (a *BaseAgent) popPendingOps() (opsToApply []*agentOperation) { +func (a *Agent) popPendingOps() (opsToApply []*agentOperation) { a.pendingOpsMu.Lock() defer a.pendingOpsMu.Unlock() @@ -260,7 +233,7 @@ func (a *BaseAgent) popPendingOps() (opsToApply []*agentOperation) { } // sendSync needs to be called with a.pendingOpsMu held. -func (a *BaseAgent) sendSync(ctx context.Context) (bool, error) { +func (a *Agent) sendSync(ctx context.Context) (bool, error) { var adding, removing, running []model.TableID for _, op := range a.tableOperations { if !op.IsDelete { @@ -291,7 +264,7 @@ func (a *BaseAgent) sendSync(ctx context.Context) (bool, error) { // processOperations tries to make progress on each pending table operations. // It queries the executor for the current status of each table. -func (a *BaseAgent) processOperations(ctx context.Context) error { +func (a *Agent) processOperations(ctx context.Context) error { for tableID, op := range a.tableOperations { switch op.status { case operationReceived: @@ -343,7 +316,7 @@ func (a *BaseAgent) processOperations(ctx context.Context) error { return nil } -func (a *BaseAgent) sendCheckpoint(ctx context.Context) error { +func (a *Agent) sendCheckpoint(ctx context.Context) error { checkpointProvider := func() (checkpointTs, resolvedTs model.Ts, ok bool) { // We cannot have a meaningful checkpoint for a processor running NO table. if len(a.executor.GetAllCurrentTables()) == 0 { @@ -363,13 +336,13 @@ func (a *BaseAgent) sendCheckpoint(ctx context.Context) error { // OnOwnerDispatchedTask should be called when the Owner sent a new dispatched task. // The Processor is responsible for calling this function when appropriate. -func (a *BaseAgent) OnOwnerDispatchedTask( +func (a *Agent) OnOwnerDispatchedTask( ownerCaptureID model.CaptureID, ownerRev int64, tableID model.TableID, startTs model.Ts, isDelete bool, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, ) { if !a.updateOwnerInfo(ownerCaptureID, ownerRev) { a.logger.Info("task from stale owner ignored", @@ -402,7 +375,7 @@ func (a *BaseAgent) OnOwnerDispatchedTask( // // ownerRev is the revision number generated by the election mechanism to // indicate the order in which owners are elected. -func (a *BaseAgent) OnOwnerAnnounce( +func (a *Agent) OnOwnerAnnounce( ownerCaptureID model.CaptureID, ownerRev int64, ) { @@ -428,7 +401,7 @@ func (a *BaseAgent) OnOwnerAnnounce( // // ownerCaptureID: the incoming owner's capture ID // ownerRev: the incoming owner's revision as generated by Etcd election. -func (a *BaseAgent) updateOwnerInfo(ownerCaptureID model.CaptureID, ownerRev int64) bool { +func (a *Agent) updateOwnerInfo(ownerCaptureID model.CaptureID, ownerRev int64) bool { a.ownerInfoMu.Lock() defer a.ownerInfoMu.Unlock() @@ -480,14 +453,14 @@ func (a *BaseAgent) updateOwnerInfo(ownerCaptureID model.CaptureID, ownerRev int return true } -func (a *BaseAgent) currentOwner() (model.CaptureID, int64 /* revision */) { +func (a *Agent) currentOwner() (model.CaptureID, int64 /* revision */) { a.ownerInfoMu.RLock() defer a.ownerInfoMu.RUnlock() return a.ownerInfo.OwnerCaptureID, a.ownerInfo.OwnerRev } -func (a *BaseAgent) resetEpoch() { +func (a *Agent) resetEpoch() { a.epochMu.Lock() defer a.epochMu.Unlock() @@ -497,7 +470,7 @@ func (a *BaseAgent) resetEpoch() { a.epoch = uuid.New().String() } -func (a *BaseAgent) getEpoch() model.ProcessorEpoch { +func (a *Agent) getEpoch() protocol.ProcessorEpoch { a.epochMu.RLock() defer a.epochMu.RUnlock() @@ -506,6 +479,6 @@ func (a *BaseAgent) getEpoch() model.ProcessorEpoch { // CurrentEpoch is a public function used in unit tests for // checking epoch-related invariants. -func (a *BaseAgent) CurrentEpoch() model.ProcessorEpoch { +func (a *Agent) CurrentEpoch() protocol.ProcessorEpoch { return a.getEpoch() } diff --git a/cdc/scheduler/agent_mock.go b/cdc/scheduler/internal/base/agent_mock.go similarity index 79% rename from cdc/scheduler/agent_mock.go rename to cdc/scheduler/internal/base/agent_mock.go index 193f1c4ec7a..6c760f7d8bb 100644 --- a/cdc/scheduler/agent_mock.go +++ b/cdc/scheduler/internal/base/agent_mock.go @@ -11,14 +11,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "testing" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/zap" @@ -30,32 +31,39 @@ type MockProcessorMessenger struct { } // FinishTableOperation marks this function as being called. -func (m *MockProcessorMessenger) FinishTableOperation(ctx cdcContext.Context, tableID model.TableID, epoch model.ProcessorEpoch) (bool, error) { +func (m *MockProcessorMessenger) FinishTableOperation( + ctx context.Context, tableID model.TableID, epoch protocol.ProcessorEpoch, +) (bool, error) { args := m.Called(ctx, tableID, epoch) return args.Bool(0), args.Error(1) } // SyncTaskStatuses marks this function as being called. -func (m *MockProcessorMessenger) SyncTaskStatuses(ctx cdcContext.Context, epoch model.ProcessorEpoch, adding, removing, running []model.TableID) (bool, error) { +func (m *MockProcessorMessenger) SyncTaskStatuses( + ctx context.Context, epoch protocol.ProcessorEpoch, + adding, removing, running []model.TableID, +) (bool, error) { args := m.Called(ctx, epoch, running, adding, removing) return args.Bool(0), args.Error(1) } // SendCheckpoint marks this function as being called. -func (m *MockProcessorMessenger) SendCheckpoint(ctx cdcContext.Context, checkpointTs model.Ts, resolvedTs model.Ts) (bool, error) { +func (m *MockProcessorMessenger) SendCheckpoint( + ctx context.Context, checkpointTs model.Ts, resolvedTs model.Ts, +) (bool, error) { args := m.Called(ctx, checkpointTs, resolvedTs) return args.Bool(0), args.Error(1) } // Barrier marks this function as being called. -func (m *MockProcessorMessenger) Barrier(ctx cdcContext.Context) (done bool) { +func (m *MockProcessorMessenger) Barrier(ctx context.Context) (done bool) { args := m.Called(ctx) return args.Bool(0) } // OnOwnerChanged marks this function as being called. func (m *MockProcessorMessenger) OnOwnerChanged( - ctx cdcContext.Context, + ctx context.Context, newOwnerCaptureID model.CaptureID, newOwnerRev int64, ) { @@ -74,7 +82,9 @@ type mockCheckpointSender struct { } // SendCheckpoint sends a checkpoint. -func (s *mockCheckpointSender) SendCheckpoint(_ cdcContext.Context, provider checkpointProviderFunc) error { +func (s *mockCheckpointSender) SendCheckpoint( + _ context.Context, provider checkpointProviderFunc, +) error { checkpointTs, resolvedTs, ok := provider() if !ok { return nil @@ -109,7 +119,9 @@ func NewMockTableExecutor(t *testing.T) *MockTableExecutor { } // AddTable adds a table to the executor. -func (e *MockTableExecutor) AddTable(ctx cdcContext.Context, tableID model.TableID, startTs model.Ts) (bool, error) { +func (e *MockTableExecutor) AddTable( + ctx context.Context, tableID model.TableID, startTs model.Ts, +) (bool, error) { log.Info("AddTable", zap.Int64("tableID", tableID)) require.NotContains(e.t, e.Adding, tableID) require.NotContains(e.t, e.Running, tableID) @@ -123,7 +135,7 @@ func (e *MockTableExecutor) AddTable(ctx cdcContext.Context, tableID model.Table } // RemoveTable removes a table from the executor. -func (e *MockTableExecutor) RemoveTable(ctx cdcContext.Context, tableID model.TableID) (bool, error) { +func (e *MockTableExecutor) RemoveTable(ctx context.Context, tableID model.TableID) (bool, error) { log.Info("RemoveTable", zap.Int64("tableID", tableID)) args := e.Called(ctx, tableID) require.Contains(e.t, e.Running, tableID) @@ -134,13 +146,13 @@ func (e *MockTableExecutor) RemoveTable(ctx cdcContext.Context, tableID model.Ta } // IsAddTableFinished determines if the table has been added. -func (e *MockTableExecutor) IsAddTableFinished(ctx cdcContext.Context, tableID model.TableID) bool { +func (e *MockTableExecutor) IsAddTableFinished(ctx context.Context, tableID model.TableID) bool { _, ok := e.Running[tableID] return ok } // IsRemoveTableFinished determines if the table has been removed. -func (e *MockTableExecutor) IsRemoveTableFinished(ctx cdcContext.Context, tableID model.TableID) bool { +func (e *MockTableExecutor) IsRemoveTableFinished(ctx context.Context, tableID model.TableID) bool { _, ok := e.Removing[tableID] return !ok } diff --git a/cdc/scheduler/agent_test.go b/cdc/scheduler/internal/base/agent_test.go similarity index 93% rename from cdc/scheduler/agent_test.go rename to cdc/scheduler/internal/base/agent_test.go index c7dbb1b0bc5..e62fcd33e8a 100644 --- a/cdc/scheduler/agent_test.go +++ b/cdc/scheduler/internal/base/agent_test.go @@ -11,28 +11,30 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "testing" "github.com/pingcap/tiflow/cdc/model" - cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" ) // read only -var agentConfigForTesting = &BaseAgentConfig{SendCheckpointTsInterval: 0} +var agentConfigForTesting = &AgentConfig{SendCheckpointTsInterval: 0} func TestAgentAddTable(t *testing.T) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} agent := NewBaseAgent(model.DefaultChangeFeedID("test-cf"), executor, messenger, agentConfigForTesting) - var epoch model.ProcessorEpoch + var epoch protocol.ProcessorEpoch messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). Return(true, nil). Run(func(args mock.Arguments) { @@ -77,7 +79,8 @@ func TestAgentAddTable(t *testing.T) { } func TestAgentRemoveTable(t *testing.T) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() executor := NewMockTableExecutor(t) executor.Running[model.TableID(1)] = struct{}{} @@ -88,7 +91,7 @@ func TestAgentRemoveTable(t *testing.T) { executor, messenger, agentConfigForTesting) agent.OnOwnerAnnounce("capture-2", 1) - var epoch model.ProcessorEpoch + var epoch protocol.ProcessorEpoch messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), []model.TableID{1, 2}, []model.TableID(nil), []model.TableID(nil)). Return(true, nil). Run(func(args mock.Arguments) { @@ -149,14 +152,15 @@ func TestAgentRemoveTable(t *testing.T) { } func TestAgentOwnerChangedWhileAddingTable(t *testing.T) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} agent := NewBaseAgent(model.DefaultChangeFeedID("test-cf"), executor, messenger, agentConfigForTesting) - var epoch model.ProcessorEpoch + var epoch protocol.ProcessorEpoch messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), @@ -211,7 +215,8 @@ func TestAgentOwnerChangedWhileAddingTable(t *testing.T) { } func TestAgentReceiveFromStaleOwner(t *testing.T) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} @@ -219,7 +224,7 @@ func TestAgentReceiveFromStaleOwner(t *testing.T) { executor, messenger, agentConfigForTesting) agent.checkpointSender = &mockCheckpointSender{} - var epoch model.ProcessorEpoch + var epoch protocol.ProcessorEpoch messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). Return(true, nil).Run(func(args mock.Arguments) { @@ -258,7 +263,8 @@ func TestAgentReceiveFromStaleOwner(t *testing.T) { } func TestOwnerMismatchShouldPanic(t *testing.T) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} @@ -287,7 +293,8 @@ func TestOwnerMismatchShouldPanic(t *testing.T) { } func TestIgnoreStaleEpoch(t *testing.T) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() executor := NewMockTableExecutor(t) messenger := &MockProcessorMessenger{} @@ -295,7 +302,7 @@ func TestIgnoreStaleEpoch(t *testing.T) { executor, messenger, agentConfigForTesting) agent.checkpointSender = &mockCheckpointSender{} - var epoch, newEpoch model.ProcessorEpoch + var epoch, newEpoch protocol.ProcessorEpoch messenger.On("SyncTaskStatuses", mock.Anything, mock.AnythingOfType("string"), []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)). Return(true, nil).Run(func(args mock.Arguments) { diff --git a/cdc/scheduler/balancer.go b/cdc/scheduler/internal/base/balancer.go similarity index 98% rename from cdc/scheduler/balancer.go rename to cdc/scheduler/internal/base/balancer.go index 1c2e99855de..5f2c9f1c82b 100644 --- a/cdc/scheduler/balancer.go +++ b/cdc/scheduler/internal/base/balancer.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( "math" @@ -19,7 +19,7 @@ import ( "time" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" "go.uber.org/zap" ) diff --git a/cdc/scheduler/balancer_test.go b/cdc/scheduler/internal/base/balancer_test.go similarity index 98% rename from cdc/scheduler/balancer_test.go rename to cdc/scheduler/internal/base/balancer_test.go index 24026ff702c..8cf4e1c8056 100644 --- a/cdc/scheduler/balancer_test.go +++ b/cdc/scheduler/internal/base/balancer_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( "testing" @@ -19,7 +19,7 @@ import ( "github.com/facebookgo/subset" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" "github.com/stretchr/testify/require" "go.uber.org/zap" ) diff --git a/cdc/scheduler/checkpoint_sender.go b/cdc/scheduler/internal/base/checkpoint_sender.go similarity index 98% rename from cdc/scheduler/checkpoint_sender.go rename to cdc/scheduler/internal/base/checkpoint_sender.go index db425fda2d3..2d3e145aa25 100644 --- a/cdc/scheduler/checkpoint_sender.go +++ b/cdc/scheduler/internal/base/checkpoint_sender.go @@ -11,15 +11,15 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "time" "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/context" "go.uber.org/zap" ) diff --git a/cdc/scheduler/checkpoint_sender_test.go b/cdc/scheduler/internal/base/checkpoint_sender_test.go similarity index 99% rename from cdc/scheduler/checkpoint_sender_test.go rename to cdc/scheduler/internal/base/checkpoint_sender_test.go index 2c05966e8ce..1c0ffabf669 100644 --- a/cdc/scheduler/checkpoint_sender_test.go +++ b/cdc/scheduler/internal/base/checkpoint_sender_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( "testing" diff --git a/cdc/scheduler/info_provider.go b/cdc/scheduler/internal/base/info_provider.go similarity index 73% rename from cdc/scheduler/info_provider.go rename to cdc/scheduler/internal/base/info_provider.go index e4bf56f3580..8b2eea96867 100644 --- a/cdc/scheduler/info_provider.go +++ b/cdc/scheduler/internal/base/info_provider.go @@ -11,36 +11,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" ) -// InfoProvider is the interface to get information about the internal states of the scheduler. -// We need this interface so that we can provide the information through HTTP API. -type InfoProvider interface { - // GetTaskStatuses returns the task statuses. - GetTaskStatuses() (map[model.CaptureID]*model.TaskStatus, error) - - // GetTaskPositions returns the task positions. - GetTaskPositions() (map[model.CaptureID]*model.TaskPosition, error) - - // GetTotalTableCounts returns the number of tables associated - // with each capture. - GetTotalTableCounts() map[model.CaptureID]int - - // GetPendingTableCounts returns the number of tables in a non-ready - // status (Adding & Removing) associated with each capture. - GetPendingTableCounts() map[model.CaptureID]int -} - // GetTaskStatuses implements InfoProvider for BaseScheduleDispatcher. // Complexity Note: This function has O(#tables) cost. USE WITH CARE. // Functions with cost O(#tables) are NOT recommended for regular metrics // collection. -func (s *BaseScheduleDispatcher) GetTaskStatuses() (map[model.CaptureID]*model.TaskStatus, error) { +func (s *ScheduleDispatcher) GetTaskStatuses() (map[model.CaptureID]*model.TaskStatus, error) { s.mu.Lock() defer s.mu.Unlock() @@ -86,7 +68,7 @@ func (s *BaseScheduleDispatcher) GetTaskStatuses() (map[model.CaptureID]*model.T // GetTaskPositions implements InfoProvider for BaseScheduleDispatcher. // Complexity Note: This function has O(#captures) cost. -func (s *BaseScheduleDispatcher) GetTaskPositions() (map[model.CaptureID]*model.TaskPosition, error) { +func (s *ScheduleDispatcher) GetTaskPositions() (map[model.CaptureID]*model.TaskPosition, error) { s.mu.Lock() defer s.mu.Unlock() @@ -103,7 +85,7 @@ func (s *BaseScheduleDispatcher) GetTaskPositions() (map[model.CaptureID]*model. // GetTotalTableCounts implements InfoProvider for BaseScheduleDispatcher. // Complexity Note: This function has O(#captures) cost. -func (s *BaseScheduleDispatcher) GetTotalTableCounts() map[model.CaptureID]int { +func (s *ScheduleDispatcher) GetTotalTableCounts() map[model.CaptureID]int { s.mu.Lock() defer s.mu.Unlock() @@ -116,7 +98,7 @@ func (s *BaseScheduleDispatcher) GetTotalTableCounts() map[model.CaptureID]int { // GetPendingTableCounts implements InfoProvider for BaseScheduleDispatcher. // Complexity Note: This function has O(#captures) cost. -func (s *BaseScheduleDispatcher) GetPendingTableCounts() map[model.CaptureID]int { +func (s *ScheduleDispatcher) GetPendingTableCounts() map[model.CaptureID]int { s.mu.Lock() defer s.mu.Unlock() diff --git a/cdc/scheduler/info_provider_test.go b/cdc/scheduler/internal/base/info_provider_test.go similarity index 92% rename from cdc/scheduler/info_provider_test.go rename to cdc/scheduler/internal/base/info_provider_test.go index 891302c12a6..a8309205697 100644 --- a/cdc/scheduler/info_provider_test.go +++ b/cdc/scheduler/internal/base/info_provider_test.go @@ -11,20 +11,21 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( "testing" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" + "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" "github.com/stretchr/testify/require" ) // Asserts that BaseSchedulerDispatcher implements InfoProvider interface. -var _ InfoProvider = (*BaseScheduleDispatcher)(nil) +var _ internal.InfoProvider = (*ScheduleDispatcher)(nil) -func injectSchedulerStateForInfoProviderTest(dispatcher *BaseScheduleDispatcher) { +func injectSchedulerStateForInfoProviderTest(dispatcher *ScheduleDispatcher) { dispatcher.captureStatus = map[model.CaptureID]*captureStatus{ "capture-1": { SyncStatus: captureSyncFinished, diff --git a/cdc/scheduler/move_table_manager.go b/cdc/scheduler/internal/base/move_table_manager.go similarity index 99% rename from cdc/scheduler/move_table_manager.go rename to cdc/scheduler/internal/base/move_table_manager.go index d7c80f97ec7..368615aa8ec 100644 --- a/cdc/scheduler/move_table_manager.go +++ b/cdc/scheduler/internal/base/move_table_manager.go @@ -11,14 +11,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "sync" "github.com/pingcap/errors" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/context" ) // Design Notes: diff --git a/cdc/scheduler/move_table_manager_test.go b/cdc/scheduler/internal/base/move_table_manager_test.go similarity index 94% rename from cdc/scheduler/move_table_manager_test.go rename to cdc/scheduler/internal/base/move_table_manager_test.go index 213385b2378..8a68162cb97 100644 --- a/cdc/scheduler/move_table_manager_test.go +++ b/cdc/scheduler/internal/base/move_table_manager_test.go @@ -11,14 +11,14 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "fmt" "testing" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/pkg/context" "github.com/stretchr/testify/require" ) @@ -39,7 +39,7 @@ func TestMoveTableManagerBasics(t *testing.T) { ok = m.Add(2, "capture-2-1") require.False(t, ok) - ctx := context.NewBackendContext4Test(false) + ctx := context.Background() // Test 4: Remove one table var removedTable model.TableID ok, err := m.DoRemove(ctx, func(ctx context.Context, tableID model.TableID, _ model.CaptureID) (result removeTableResult, err error) { @@ -92,7 +92,7 @@ func TestMoveTableManagerCaptureRemoved(t *testing.T) { m.OnCaptureRemoved("capture-2") - ctx := context.NewBackendContext4Test(false) + ctx := context.Background() var count int ok, err := m.DoRemove(ctx, func(ctx context.Context, tableID model.TableID, target model.CaptureID) (result removeTableResult, err error) { @@ -116,7 +116,7 @@ func TestMoveTableManagerGiveUp(t *testing.T) { ok = m.Add(2, "capture-2") require.True(t, ok) - ctx := context.NewBackendContext4Test(false) + ctx := context.Background() ok, err := m.DoRemove(ctx, func(ctx context.Context, tableID model.TableID, target model.CaptureID) (result removeTableResult, err error) { if tableID == 1 { diff --git a/cdc/owner/scheduler.go b/cdc/scheduler/internal/base/owner_scheduler.go similarity index 66% rename from cdc/owner/scheduler.go rename to cdc/scheduler/internal/base/owner_scheduler.go index 0bce4e4425b..5d9f617cdde 100644 --- a/cdc/owner/scheduler.go +++ b/cdc/scheduler/internal/base/owner_scheduler.go @@ -11,50 +11,27 @@ // See the License for the specific language governing permissions and // limitations under the License. -package owner +package base import ( + "context" "sync/atomic" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - pscheduler "github.com/pingcap/tiflow/cdc/scheduler" - "github.com/pingcap/tiflow/pkg/context" + sched "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" cerror "github.com/pingcap/tiflow/pkg/errors" - "github.com/pingcap/tiflow/pkg/orchestrator" "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/version" "go.uber.org/zap" ) -// scheduler is an interface for scheduling tables. -// Since in our design, we do not record checkpoints per table, -// how we calculate the global watermarks (checkpoint-ts and resolved-ts) -// is heavily coupled with how tables are scheduled. -// That is why we have a scheduler interface that also reports the global watermarks. -type scheduler interface { - // Tick is called periodically from the owner, and returns - // updated global watermarks. - Tick( - ctx context.Context, - state *orchestrator.ChangefeedReactorState, - currentTables []model.TableID, - captures map[model.CaptureID]*model.CaptureInfo, - ) (newCheckpointTs, newResolvedTs model.Ts, err error) - - // MoveTable is used to trigger manual table moves. - MoveTable(tableID model.TableID, target model.CaptureID) - - // Rebalance is used to trigger manual workload rebalances. - Rebalance() - - // Close closes the scheduler and releases resources. - Close(ctx context.Context) -} - -type schedulerV2 struct { - *pscheduler.BaseScheduleDispatcher +// SchedulerV2 schedules tables through P2P. +type SchedulerV2 struct { + *ScheduleDispatcher + ownerRevision int64 messageServer *p2p.MessageServer messageRouter p2p.MessageRouter @@ -65,21 +42,23 @@ type schedulerV2 struct { stats *schedulerStats } -// NewSchedulerV2 creates a new schedulerV2 +// NewSchedulerV2 creates a new SchedulerV2 func NewSchedulerV2( ctx context.Context, changeFeedID model.ChangeFeedID, checkpointTs model.Ts, messageServer *p2p.MessageServer, messageRouter p2p.MessageRouter, -) (*schedulerV2, error) { - ret := &schedulerV2{ + ownerRevision int64, +) (*SchedulerV2, error) { + ret := &SchedulerV2{ changeFeedID: changeFeedID, messageServer: messageServer, messageRouter: messageRouter, stats: &schedulerStats{}, + ownerRevision: ownerRevision, } - ret.BaseScheduleDispatcher = pscheduler.NewBaseScheduleDispatcher(changeFeedID, ret, checkpointTs) + ret.ScheduleDispatcher = NewBaseScheduleDispatcher(changeFeedID, ret, checkpointTs) if err := ret.registerPeerMessageHandlers(ctx); err != nil { return nil, err } @@ -87,47 +66,32 @@ func NewSchedulerV2( return ret, nil } -// newSchedulerV2FromCtx creates a new schedulerV2 from context. -// This function is factored out to facilitate unit testing. -func newSchedulerV2FromCtx(ctx context.Context, startTs uint64) (scheduler, error) { - changeFeedID := ctx.ChangefeedVars().ID - messageServer := ctx.GlobalVars().MessageServer - messageRouter := ctx.GlobalVars().MessageRouter - ret, err := NewSchedulerV2(ctx, changeFeedID, startTs, messageServer, messageRouter) - if err != nil { - return nil, errors.Trace(err) - } - return ret, nil -} - -func newScheduler(ctx context.Context, startTs uint64) (scheduler, error) { - return newSchedulerV2FromCtx(ctx, startTs) -} - -func (s *schedulerV2) Tick( +// Tick implements the interface ScheduleDispatcher. +func (s *SchedulerV2) Tick( ctx context.Context, - state *orchestrator.ChangefeedReactorState, + checkpointTs model.Ts, currentTables []model.TableID, captures map[model.CaptureID]*model.CaptureInfo, ) (checkpoint, resolvedTs model.Ts, err error) { if err := s.checkForHandlerErrors(ctx); err != nil { - return pscheduler.CheckpointCannotProceed, pscheduler.CheckpointCannotProceed, errors.Trace(err) + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, errors.Trace(err) } - return s.BaseScheduleDispatcher.Tick(ctx, state.Status.CheckpointTs, currentTables, captures) + return s.ScheduleDispatcher.Tick(ctx, checkpointTs, currentTables, captures) } -func (s *schedulerV2) DispatchTable( +// DispatchTable implements the interface ScheduleDispatcherCommunicator. +func (s *SchedulerV2) DispatchTable( ctx context.Context, changeFeedID model.ChangeFeedID, tableID model.TableID, startTs model.Ts, captureID model.CaptureID, isDelete bool, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, ) (done bool, err error) { - topic := model.DispatchTableTopic(changeFeedID) - message := &model.DispatchTableMessage{ - OwnerRev: ctx.GlobalVars().OwnerRevision, + topic := protocol.DispatchTableTopic(changeFeedID) + message := &protocol.DispatchTableMessage{ + OwnerRev: s.ownerRevision, ID: tableID, StartTs: startTs, IsDelete: isDelete, @@ -162,14 +126,15 @@ func (s *schedulerV2) DispatchTable( return true, nil } -func (s *schedulerV2) Announce( +// Announce implements the interface ScheduleDispatcherCommunicator. +func (s *SchedulerV2) Announce( ctx context.Context, changeFeedID model.ChangeFeedID, captureID model.CaptureID, ) (done bool, err error) { - topic := model.AnnounceTopic(changeFeedID) - message := &model.AnnounceMessage{ - OwnerRev: ctx.GlobalVars().OwnerRevision, + topic := protocol.AnnounceTopic(changeFeedID) + message := &protocol.AnnounceMessage{ + OwnerRev: s.ownerRevision, OwnerVersion: version.ReleaseSemver(), } @@ -201,7 +166,7 @@ func (s *schedulerV2) Announce( return true, nil } -func (s *schedulerV2) getClient(target model.CaptureID) (*p2p.MessageClient, bool) { +func (s *SchedulerV2) getClient(target model.CaptureID) (*p2p.MessageClient, bool) { client := s.messageRouter.GetClient(target) if client == nil { log.Warn("scheduler: no message client found, retry later", @@ -211,13 +176,14 @@ func (s *schedulerV2) getClient(target model.CaptureID) (*p2p.MessageClient, boo return client, true } -func (s *schedulerV2) trySendMessage( +func (s *SchedulerV2) trySendMessage( ctx context.Context, target model.CaptureID, topic p2p.Topic, value interface{}, ) (bool, error) { - // TODO (zixiong): abstract this function out together with the similar method in cdc/processor/agent.go + // TODO (zixiong): abstract this function out together with the similar method in + // cdc/scheduler/processor_agent.go // We probably need more advanced logic to handle and mitigate complex failure situations. client, ok := s.getClient(target) @@ -244,14 +210,15 @@ func (s *schedulerV2) trySendMessage( return true, nil } -func (s *schedulerV2) Close(ctx context.Context) { +// Close implements the interface ScheduleDispatcher. +func (s *SchedulerV2) Close(ctx context.Context) { log.Debug("scheduler closed", zap.String("namespace", s.changeFeedID.Namespace), zap.String("changefeedID", s.changeFeedID.ID)) s.deregisterPeerMessageHandlers(ctx) } -func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret error) { +func (s *SchedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret error) { defer func() { if ret != nil { s.deregisterPeerMessageHandlers(ctx) @@ -260,10 +227,10 @@ func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret erro errCh, err := s.messageServer.SyncAddHandler( ctx, - model.DispatchTableResponseTopic(s.changeFeedID), - &model.DispatchTableResponseMessage{}, + protocol.DispatchTableResponseTopic(s.changeFeedID), + &protocol.DispatchTableResponseMessage{}, func(sender string, messageI interface{}) error { - message := messageI.(*model.DispatchTableResponseMessage) + message := messageI.(*protocol.DispatchTableResponseMessage) s.stats.RecordDispatchResponse() s.OnAgentFinishedTableOperation(sender, message.ID, message.Epoch) return nil @@ -275,10 +242,10 @@ func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret erro errCh, err = s.messageServer.SyncAddHandler( ctx, - model.SyncTopic(s.changeFeedID), - &model.SyncMessage{}, + protocol.SyncTopic(s.changeFeedID), + &protocol.SyncMessage{}, func(sender string, messageI interface{}) error { - message := messageI.(*model.SyncMessage) + message := messageI.(*protocol.SyncMessage) s.stats.RecordSync() s.OnAgentSyncTaskStatuses( sender, @@ -295,10 +262,10 @@ func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret erro errCh, err = s.messageServer.SyncAddHandler( ctx, - model.CheckpointTopic(s.changeFeedID), - &model.CheckpointMessage{}, + protocol.CheckpointTopic(s.changeFeedID), + &protocol.CheckpointMessage{}, func(sender string, messageI interface{}) error { - message := messageI.(*model.CheckpointMessage) + message := messageI.(*protocol.CheckpointMessage) s.stats.RecordCheckpoint() s.OnAgentCheckpoint(sender, message.CheckpointTs, message.ResolvedTs) return nil @@ -311,30 +278,30 @@ func (s *schedulerV2) registerPeerMessageHandlers(ctx context.Context) (ret erro return nil } -func (s *schedulerV2) deregisterPeerMessageHandlers(ctx context.Context) { +func (s *SchedulerV2) deregisterPeerMessageHandlers(ctx context.Context) { err := s.messageServer.SyncRemoveHandler( ctx, - model.DispatchTableResponseTopic(s.changeFeedID)) + protocol.DispatchTableResponseTopic(s.changeFeedID)) if err != nil { log.Error("failed to remove peer message handler", zap.Error(err)) } err = s.messageServer.SyncRemoveHandler( ctx, - model.SyncTopic(s.changeFeedID)) + protocol.SyncTopic(s.changeFeedID)) if err != nil { log.Error("failed to remove peer message handler", zap.Error(err)) } err = s.messageServer.SyncRemoveHandler( ctx, - model.CheckpointTopic(s.changeFeedID)) + protocol.CheckpointTopic(s.changeFeedID)) if err != nil { log.Error("failed to remove peer message handler", zap.Error(err)) } } -func (s *schedulerV2) checkForHandlerErrors(ctx context.Context) error { +func (s *SchedulerV2) checkForHandlerErrors(ctx context.Context) error { for _, errCh := range s.handlerErrChs { select { case <-ctx.Done(): diff --git a/cdc/owner/scheduler_test.go b/cdc/scheduler/internal/base/owner_scheduler_test.go similarity index 63% rename from cdc/owner/scheduler_test.go rename to cdc/scheduler/internal/base/owner_scheduler_test.go index 8a017559291..3124d9d1677 100644 --- a/cdc/owner/scheduler_test.go +++ b/cdc/scheduler/internal/base/owner_scheduler_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package owner +package base import ( "context" @@ -22,9 +22,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - pscheduler "github.com/pingcap/tiflow/cdc/scheduler" - cdcContext "github.com/pingcap/tiflow/pkg/context" - "github.com/pingcap/tiflow/pkg/orchestrator" + "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/version" "github.com/stretchr/testify/require" @@ -45,13 +44,9 @@ func TestSchedulerBasics(t *testing.T) { _ = failpoint.Disable("github.com/pingcap/tiflow/pkg/p2p/ClientInjectClosed") }() - stdCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() - ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{ - OwnerRevision: 1, - }) - mockCluster := p2p.NewMockCluster(t, numNodes) mockCaptures := map[model.CaptureID]*model.CaptureInfo{} @@ -69,20 +64,16 @@ func TestSchedulerBasics(t *testing.T) { model.DefaultChangeFeedID("cf-1"), 1000, mockOwnerNode.Server, - mockOwnerNode.Router) + mockOwnerNode.Router, + 1) require.NoError(t, err) for atomic.LoadInt64(&sched.stats.AnnounceSentCount) < numNodes { - checkpointTs, resolvedTs, err := sched.Tick(ctx, &orchestrator.ChangefeedReactorState{ - ID: model.DefaultChangeFeedID("cf-1"), - Status: &model.ChangeFeedStatus{ - ResolvedTs: 1000, - CheckpointTs: 1000, - }, - }, []model.TableID{1, 2, 3}, mockCaptures) + checkpointTs, resolvedTs, err := sched.Tick( + ctx, 1000, []model.TableID{1, 2, 3}, mockCaptures) require.NoError(t, err) - require.Equal(t, pscheduler.CheckpointCannotProceed, checkpointTs) - require.Equal(t, pscheduler.CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) } announceCh := receiveToChannels( @@ -90,15 +81,15 @@ func TestSchedulerBasics(t *testing.T) { t, mockOwnerNode.ID, mockCluster, - model.AnnounceTopic(model.DefaultChangeFeedID("cf-1")), - &model.AnnounceMessage{}) + protocol.AnnounceTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.AnnounceMessage{}) dispatchCh := receiveToChannels( ctx, t, mockOwnerNode.ID, mockCluster, - model.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), - &model.DispatchTableMessage{}) + protocol.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.DispatchTableMessage{}) for id, ch := range announceCh { var msg interface{} @@ -108,16 +99,16 @@ func TestSchedulerBasics(t *testing.T) { case msg = <-ch: } - require.IsType(t, &model.AnnounceMessage{}, msg) - require.Equal(t, &model.AnnounceMessage{ + require.IsType(t, &protocol.AnnounceMessage{}, msg) + require.Equal(t, &protocol.AnnounceMessage{ OwnerRev: 1, OwnerVersion: version.ReleaseSemver(), }, msg) _, err := mockCluster.Nodes[id].Router.GetClient(mockOwnerNode.ID).SendMessage( ctx, - model.SyncTopic(model.DefaultChangeFeedID("cf-1")), - &model.SyncMessage{ + protocol.SyncTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), }) require.NoError(t, err) @@ -128,16 +119,11 @@ func TestSchedulerBasics(t *testing.T) { }, 5*time.Second, 100*time.Millisecond) for atomic.LoadInt64(&sched.stats.DispatchSentCount) < numNodes { - checkpointTs, resolvedTs, err := sched.Tick(ctx, &orchestrator.ChangefeedReactorState{ - ID: model.DefaultChangeFeedID("cf-1"), - Status: &model.ChangeFeedStatus{ - ResolvedTs: 1000, - CheckpointTs: 1000, - }, - }, []model.TableID{1, 2, 3}, mockCaptures) + checkpointTs, resolvedTs, err := sched.Tick( + ctx, 1000, []model.TableID{1, 2, 3}, mockCaptures) require.NoError(t, err) - require.Equal(t, pscheduler.CheckpointCannotProceed, checkpointTs) - require.Equal(t, pscheduler.CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) } log.Info("Tables have been dispatched") @@ -149,16 +135,16 @@ func TestSchedulerBasics(t *testing.T) { case msg = <-ch: } - require.IsType(t, &model.DispatchTableMessage{}, msg) - dispatchTableMessage := msg.(*model.DispatchTableMessage) + require.IsType(t, &protocol.DispatchTableMessage{}, msg) + dispatchTableMessage := msg.(*protocol.DispatchTableMessage) require.Equal(t, int64(1), dispatchTableMessage.OwnerRev) require.False(t, dispatchTableMessage.IsDelete) require.Contains(t, []model.TableID{1, 2, 3}, dispatchTableMessage.ID) _, err := mockCluster.Nodes[id].Router.GetClient(mockOwnerNode.ID).SendMessage( ctx, - model.DispatchTableResponseTopic(model.DefaultChangeFeedID("cf-1")), - &model.DispatchTableResponseMessage{ + protocol.DispatchTableResponseTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.DispatchTableResponseMessage{ ID: dispatchTableMessage.ID, }) require.NoError(t, err) @@ -168,22 +154,17 @@ func TestSchedulerBasics(t *testing.T) { return atomic.LoadInt64(&sched.stats.DispatchResponseReceiveCount) == 3 }, 5*time.Second, 100*time.Millisecond) - checkpointTs, resolvedTs, err := sched.Tick(ctx, &orchestrator.ChangefeedReactorState{ - ID: model.DefaultChangeFeedID("cf-1"), - Status: &model.ChangeFeedStatus{ - ResolvedTs: 1000, - CheckpointTs: 1000, - }, - }, []model.TableID{1, 2, 3}, mockCaptures) + checkpointTs, resolvedTs, err := sched.Tick( + ctx, 1000, []model.TableID{1, 2, 3}, mockCaptures) require.NoError(t, err) require.Equal(t, model.Ts(1000), checkpointTs) require.Equal(t, model.Ts(1000), resolvedTs) for _, node := range mockCluster.Nodes { _, err := node.Router.GetClient(mockOwnerNode.ID). - SendMessage(ctx, model.CheckpointTopic( + SendMessage(ctx, protocol.CheckpointTopic( model.DefaultChangeFeedID("cf-1")), - &model.CheckpointMessage{ + &protocol.CheckpointMessage{ CheckpointTs: 2000, ResolvedTs: 2000, }) @@ -199,13 +180,9 @@ func TestSchedulerBasics(t *testing.T) { } func TestSchedulerNoPeer(t *testing.T) { - stdCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), 10*time.Second) defer cancel() - ctx := cdcContext.NewContext(stdCtx, &cdcContext.GlobalVars{ - OwnerRevision: 1, - }) - mockCluster := p2p.NewMockCluster(t, numNodes) mockCaptures := map[model.CaptureID]*model.CaptureInfo{} @@ -227,37 +204,28 @@ func TestSchedulerNoPeer(t *testing.T) { model.DefaultChangeFeedID("cf-1"), 1000, mockOwnerNode.Server, - mockOwnerNode.Router) + mockOwnerNode.Router, + 1) require.NoError(t, err) // Ticks the scheduler 10 times. It should not panic. for i := 0; i < 10; i++ { - checkpointTs, resolvedTs, err := sched.Tick(ctx, &orchestrator.ChangefeedReactorState{ - ID: model.DefaultChangeFeedID("cf-1"), - Status: &model.ChangeFeedStatus{ - ResolvedTs: 1000, - CheckpointTs: 1000, - }, - }, []model.TableID{1, 2, 3}, mockCaptures) + checkpointTs, resolvedTs, err := sched.Tick( + ctx, 1000, []model.TableID{1, 2, 3}, mockCaptures) require.NoError(t, err) - require.Equal(t, pscheduler.CheckpointCannotProceed, checkpointTs) - require.Equal(t, pscheduler.CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) } // Remove the node from the captureInfos. delete(mockCaptures, "dead-capture") for atomic.LoadInt64(&sched.stats.AnnounceSentCount) < numNodes { - checkpointTs, resolvedTs, err := sched.Tick(ctx, &orchestrator.ChangefeedReactorState{ - ID: model.DefaultChangeFeedID("cf-1"), - Status: &model.ChangeFeedStatus{ - ResolvedTs: 1000, - CheckpointTs: 1000, - }, - }, []model.TableID{1, 2, 3}, mockCaptures) + checkpointTs, resolvedTs, err := sched.Tick( + ctx, 1000, []model.TableID{1, 2, 3}, mockCaptures) require.NoError(t, err) - require.Equal(t, pscheduler.CheckpointCannotProceed, checkpointTs) - require.Equal(t, pscheduler.CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) } sched.Close(ctx) @@ -265,8 +233,9 @@ func TestSchedulerNoPeer(t *testing.T) { } func TestInfoProvider(t *testing.T) { - sched := scheduler(new(schedulerV2)) - _, ok := sched.(pscheduler.InfoProvider) + var sched interface{} + sched = new(SchedulerV2) + _, ok := sched.(internal.InfoProvider) require.True(t, ok) } diff --git a/cdc/processor/agent.go b/cdc/scheduler/internal/base/processor_agent.go similarity index 84% rename from cdc/processor/agent.go rename to cdc/scheduler/internal/base/processor_agent.go index 379b3cdd1ba..c49c276e4c1 100644 --- a/cdc/processor/agent.go +++ b/cdc/scheduler/internal/base/processor_agent.go @@ -11,29 +11,27 @@ // See the License for the specific language governing permissions and // limitations under the License. -package processor +package base import ( - stdContext "context" + "context" "time" - "go.uber.org/zap/zapcore" - "github.com/benbjohnson/clock" "github.com/pingcap/errors" "github.com/pingcap/log" - "go.etcd.io/etcd/client/v3/concurrency" - "go.uber.org/zap" - "golang.org/x/time/rate" - "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler" + "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" "github.com/pingcap/tiflow/pkg/config" - "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/version" + "go.etcd.io/etcd/client/v3/concurrency" + "go.uber.org/zap" + "go.uber.org/zap/zapcore" + "golang.org/x/time/rate" ) const ( @@ -43,24 +41,8 @@ const ( printWarnLogMinInterval = time.Second * 1 ) -// processorAgent is a data structure in the Processor that serves as a bridge with -// the Owner. -// -// processorAgent has a BaseAgent embedded in it, which handles the high-level logic of receiving -// commands from the Owner. It also implements ProcessorMessenger interface, which -// provides the BaseAgent with the necessary methods to send messages to the Owner. -// -// The reason for this design is to decouple the scheduling algorithm with the underlying -// RPC server/client. -// -// Note that Agent is not thread-safe, and it is not necessary for it to be thread-safe. -type processorAgent interface { - scheduler.Agent - scheduler.ProcessorMessenger -} - type agentImpl struct { - *scheduler.BaseAgent + *Agent messageServer *p2p.MessageServer messageRouter p2p.MessageRouter @@ -68,6 +50,7 @@ type agentImpl struct { changeFeed model.ChangeFeedID ownerCaptureID model.CaptureID ownerRevision int64 + etcdClient *etcd.CDCEtcdClient clock clock.Clock barrierSeqs map[p2p.Topic]p2p.Seq @@ -80,16 +63,19 @@ type agentImpl struct { handlerErrChs []<-chan error } -func newAgent( +// NewAgent returns processor agent. +func NewAgent( ctx context.Context, messageServer *p2p.MessageServer, messageRouter p2p.MessageRouter, - executor scheduler.TableExecutor, + etcdClient *etcd.CDCEtcdClient, + executor internal.TableExecutor, changeFeedID model.ChangeFeedID, -) (retVal processorAgent, err error) { +) (retVal internal.Agent, err error) { ret := &agentImpl{ messageServer: messageServer, messageRouter: messageRouter, + etcdClient: etcdClient, changeFeed: changeFeedID, @@ -108,11 +94,11 @@ func newAgent( zap.String("changefeed", changeFeedID.ID), zap.Duration("sendCheckpointTsInterval", flushInterval)) - ret.BaseAgent = scheduler.NewBaseAgent( + ret.Agent = NewBaseAgent( changeFeedID, executor, ret, - &scheduler.BaseAgentConfig{SendCheckpointTsInterval: flushInterval}) + &AgentConfig{SendCheckpointTsInterval: flushInterval}) // Note that registerPeerMessageHandlers sets handlerErrChs. if err := ret.registerPeerMessageHandlers(); err != nil { @@ -133,9 +119,9 @@ func newAgent( } }() - etcdCliCtx, cancel := stdContext.WithTimeout(ctx, getOwnerFromEtcdTimeout) + etcdCliCtx, cancel := context.WithTimeout(ctx, getOwnerFromEtcdTimeout) defer cancel() - ownerCaptureID, err := ctx.GlobalVars().EtcdClient. + ownerCaptureID, err := ret.etcdClient. GetOwnerID(etcdCliCtx, etcd.CaptureOwnerKey) if err != nil { if err != concurrency.ErrElectionNoLeader { @@ -157,7 +143,7 @@ func newAgent( zap.String("changefeed", changeFeedID.ID), zap.String("ownerID", ownerCaptureID)) - ret.ownerRevision, err = ctx.GlobalVars().EtcdClient. + ret.ownerRevision, err = ret.etcdClient. GetOwnerRevision(etcdCliCtx, ownerCaptureID) if err != nil { if cerror.ErrOwnerNotFound.Equal(err) || cerror.ErrNotOwner.Equal(err) { @@ -187,7 +173,7 @@ func (a *agentImpl) Tick(ctx context.Context) error { } } - if err := a.BaseAgent.Tick(ctx); err != nil { + if err := a.Agent.Tick(ctx); err != nil { return errors.Trace(err) } return nil @@ -196,9 +182,9 @@ func (a *agentImpl) Tick(ctx context.Context) error { func (a *agentImpl) FinishTableOperation( ctx context.Context, tableID model.TableID, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, ) (done bool, err error) { - topic := model.SyncTopic(a.changeFeed) + topic := protocol.SyncTopic(a.changeFeed) if !a.Barrier(ctx) { if _, exists := a.barrierSeqs[topic]; exists { log.L().Info("Delay sending FinishTableOperation due to pending sync", @@ -211,7 +197,7 @@ func (a *agentImpl) FinishTableOperation( } } - message := &model.DispatchTableResponseMessage{ID: tableID, Epoch: epoch} + message := &protocol.DispatchTableResponseMessage{ID: tableID, Epoch: epoch} defer func() { if err != nil { return @@ -225,7 +211,7 @@ func (a *agentImpl) FinishTableOperation( done, err = a.trySendMessage( ctx, a.ownerCaptureID, - model.DispatchTableResponseTopic(a.changeFeed), + protocol.DispatchTableResponseTopic(a.changeFeed), message) if err != nil { return false, errors.Trace(err) @@ -234,14 +220,14 @@ func (a *agentImpl) FinishTableOperation( } func (a *agentImpl) SyncTaskStatuses( - ctx context.Context, epoch model.ProcessorEpoch, adding, removing, running []model.TableID, + ctx context.Context, epoch protocol.ProcessorEpoch, adding, removing, running []model.TableID, ) (done bool, err error) { if !a.Barrier(ctx) { // The Sync message needs to be strongly ordered w.r.t. other messages. return false, nil } - message := &model.SyncMessage{ + message := &protocol.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), Epoch: epoch, Running: running, @@ -274,7 +260,7 @@ func (a *agentImpl) SyncTaskStatuses( done, err = a.trySendMessage( ctx, a.ownerCaptureID, - model.SyncTopic(a.changeFeed), + protocol.SyncTopic(a.changeFeed), message) if err != nil { return false, errors.Trace(err) @@ -287,7 +273,7 @@ func (a *agentImpl) SendCheckpoint( checkpointTs model.Ts, resolvedTs model.Ts, ) (done bool, err error) { - message := &model.CheckpointMessage{ + message := &protocol.CheckpointMessage{ CheckpointTs: checkpointTs, ResolvedTs: resolvedTs, } @@ -309,7 +295,7 @@ func (a *agentImpl) SendCheckpoint( done, err = a.trySendMessage( ctx, a.ownerCaptureID, - model.CheckpointTopic(a.changeFeed), + protocol.CheckpointTopic(a.changeFeed), message) if err != nil { return false, errors.Trace(err) @@ -408,7 +394,8 @@ func (a *agentImpl) trySendMessage( topic p2p.Topic, value interface{}, ) (bool, error) { - // TODO (zixiong): abstract this function out together with the similar method in cdc/owner/scheduler.go + // TODO (zixiong): abstract this function out together with the similar method + // in cdc/scheduler/internal/base/owner_scheduler.go // We probably need more advanced logic to handle and mitigate complex failure situations. client := a.messageRouter.GetClient(target) @@ -446,16 +433,16 @@ func (a *agentImpl) registerPeerMessageHandlers() (ret error) { } }() - ctx, cancel := stdContext.WithTimeout(stdContext.Background(), messageHandlerOperationsTimeout) + ctx, cancel := context.WithTimeout(context.Background(), messageHandlerOperationsTimeout) defer cancel() errCh, err := a.messageServer.SyncAddHandler( ctx, - model.DispatchTableTopic(a.changeFeed), - &model.DispatchTableMessage{}, + protocol.DispatchTableTopic(a.changeFeed), + &protocol.DispatchTableMessage{}, func(sender string, value interface{}) error { ownerCapture := sender - message := value.(*model.DispatchTableMessage) + message := value.(*protocol.DispatchTableMessage) a.OnOwnerDispatchedTask( ownerCapture, message.OwnerRev, @@ -472,11 +459,11 @@ func (a *agentImpl) registerPeerMessageHandlers() (ret error) { errCh, err = a.messageServer.SyncAddHandler( ctx, - model.AnnounceTopic(a.changeFeed), - &model.AnnounceMessage{}, + protocol.AnnounceTopic(a.changeFeed), + &protocol.AnnounceMessage{}, func(sender string, value interface{}) error { ownerCapture := sender - message := value.(*model.AnnounceMessage) + message := value.(*protocol.AnnounceMessage) a.OnOwnerAnnounce( ownerCapture, message.OwnerRev) @@ -490,15 +477,15 @@ func (a *agentImpl) registerPeerMessageHandlers() (ret error) { } func (a *agentImpl) deregisterPeerMessageHandlers() error { - ctx, cancel := stdContext.WithTimeout(stdContext.Background(), messageHandlerOperationsTimeout) + ctx, cancel := context.WithTimeout(context.Background(), messageHandlerOperationsTimeout) defer cancel() - err := a.messageServer.SyncRemoveHandler(ctx, model.DispatchTableTopic(a.changeFeed)) + err := a.messageServer.SyncRemoveHandler(ctx, protocol.DispatchTableTopic(a.changeFeed)) if err != nil { return errors.Trace(err) } - err = a.messageServer.SyncRemoveHandler(ctx, model.AnnounceTopic(a.changeFeed)) + err = a.messageServer.SyncRemoveHandler(ctx, protocol.AnnounceTopic(a.changeFeed)) if err != nil { return errors.Trace(err) } diff --git a/cdc/processor/agent_test.go b/cdc/scheduler/internal/base/processor_agent_test.go similarity index 83% rename from cdc/processor/agent_test.go rename to cdc/scheduler/internal/base/processor_agent_test.go index 9e1126624cb..ec03d5a3302 100644 --- a/cdc/processor/agent_test.go +++ b/cdc/scheduler/internal/base/processor_agent_test.go @@ -11,7 +11,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package processor +package base import ( "context" @@ -26,8 +26,7 @@ import ( clientv3 "go.etcd.io/etcd/client/v3" "github.com/pingcap/tiflow/cdc/model" - pscheduler "github.com/pingcap/tiflow/cdc/scheduler" - cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/version" @@ -52,15 +51,14 @@ type agentTestSuite struct { etcdClient *clientv3.Client etcdKVClient *mockEtcdKVClient - tableExecutor *pscheduler.MockTableExecutor - dispatchResponseCh chan *model.DispatchTableResponseMessage - syncCh chan *model.SyncMessage - checkpointCh chan *model.CheckpointMessage + tableExecutor *MockTableExecutor + dispatchResponseCh chan *protocol.DispatchTableResponseMessage + syncCh chan *protocol.SyncMessage + checkpointCh chan *protocol.CheckpointMessage ownerMessageClient *p2p.MessageClient ctx context.Context - cdcCtx cdcContext.Context cancel context.CancelFunc blockSyncMu sync.Mutex @@ -85,9 +83,9 @@ func newAgentTestSuite(t *testing.T) *agentTestSuite { // The channel sizes 1024 should be more than sufficient for these tests. // Full channels will result in panics to make the cases fail. - dispatchResponseCh: make(chan *model.DispatchTableResponseMessage, 1024), - syncCh: make(chan *model.SyncMessage, 1024), - checkpointCh: make(chan *model.CheckpointMessage, 1024), + dispatchResponseCh: make(chan *protocol.DispatchTableResponseMessage, 1024), + syncCh: make(chan *protocol.SyncMessage, 1024), + checkpointCh: make(chan *protocol.CheckpointMessage, 1024), ownerMessageClient: ownerMessageClient, @@ -95,14 +93,14 @@ func newAgentTestSuite(t *testing.T) *agentTestSuite { cancel: cancel, } - _, err := ownerMessageServer.SyncAddHandler(ctx, model.DispatchTableResponseTopic( + _, err := ownerMessageServer.SyncAddHandler(ctx, protocol.DispatchTableResponseTopic( model.DefaultChangeFeedID("cf-1")), - &model.DispatchTableResponseMessage{}, + &protocol.DispatchTableResponseMessage{}, func(senderID string, msg interface{}) error { require.Equal(t, processorCaptureID, senderID) - require.IsType(t, &model.DispatchTableResponseMessage{}, msg) + require.IsType(t, &protocol.DispatchTableResponseMessage{}, msg) select { - case ret.dispatchResponseCh <- msg.(*model.DispatchTableResponseMessage): + case ret.dispatchResponseCh <- msg.(*protocol.DispatchTableResponseMessage): default: require.FailNow(t, "full channel") } @@ -111,9 +109,9 @@ func newAgentTestSuite(t *testing.T) *agentTestSuite { ) require.NoError(t, err) - _, err = ownerMessageServer.SyncAddHandler(ctx, model.SyncTopic( + _, err = ownerMessageServer.SyncAddHandler(ctx, protocol.SyncTopic( model.DefaultChangeFeedID("cf-1")), - &model.SyncMessage{}, + &protocol.SyncMessage{}, func(senderID string, msg interface{}) error { ret.blockSyncMu.Lock() for ret.blockSync { @@ -122,10 +120,10 @@ func newAgentTestSuite(t *testing.T) *agentTestSuite { ret.blockSyncMu.Unlock() require.Equal(t, processorCaptureID, senderID) - require.IsType(t, &model.SyncMessage{}, msg) + require.IsType(t, &protocol.SyncMessage{}, msg) select { - case ret.syncCh <- msg.(*model.SyncMessage): + case ret.syncCh <- msg.(*protocol.SyncMessage): default: require.FailNow(t, "full channel") } @@ -134,15 +132,15 @@ func newAgentTestSuite(t *testing.T) *agentTestSuite { ) require.NoError(t, err) - _, err = ownerMessageServer.SyncAddHandler(ctx, model.CheckpointTopic( + _, err = ownerMessageServer.SyncAddHandler(ctx, protocol.CheckpointTopic( model.DefaultChangeFeedID("cf-1")), - &model.CheckpointMessage{}, + &protocol.CheckpointMessage{}, func(senderID string, msg interface{}) error { require.Equal(t, processorCaptureID, senderID) - require.IsType(t, &model.CheckpointMessage{}, msg) + require.IsType(t, &protocol.CheckpointMessage{}, msg) select { - case ret.checkpointCh <- msg.(*model.CheckpointMessage): + case ret.checkpointCh <- msg.(*protocol.CheckpointMessage): default: require.FailNow(t, "full channel") } @@ -158,17 +156,11 @@ func (s *agentTestSuite) CreateAgent(t *testing.T) (*agentImpl, error) { cdcEtcdClient := etcd.NewCDCEtcdClient(s.ctx, s.etcdClient) messageServer := s.cluster.Nodes["capture-1"].Server messageRouter := s.cluster.Nodes["capture-1"].Router - s.tableExecutor = pscheduler.NewMockTableExecutor(t) + s.tableExecutor = NewMockTableExecutor(t) - ctx := cdcContext.NewContext(s.ctx, &cdcContext.GlobalVars{ - EtcdClient: &cdcEtcdClient, - MessageServer: s.cluster.Nodes[processorCaptureID].Server, - MessageRouter: s.cluster.Nodes[processorCaptureID].Router, - }) - s.cdcCtx = ctx - - ret, err := newAgent(ctx, messageServer, messageRouter, s.tableExecutor, - model.DefaultChangeFeedID("cf-1")) + ret, err := NewAgent( + s.ctx, messageServer, messageRouter, &cdcEtcdClient, + s.tableExecutor, model.DefaultChangeFeedID("cf-1")) if err != nil { return nil, err } @@ -243,14 +235,14 @@ func TestAgentBasics(t *testing.T) { require.NoError(t, err) // Test Point 2: First tick should sync the SyncMessage. - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) select { case <-suite.ctx.Done(): require.Fail(t, "context should not be canceled") case syncMsg := <-suite.syncCh: - require.Equal(t, &model.SyncMessage{ + require.Equal(t, &protocol.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), Epoch: agent.CurrentEpoch(), Running: nil, @@ -260,8 +252,8 @@ func TestAgentBasics(t *testing.T) { } _, err = suite.ownerMessageClient.SendMessage(suite.ctx, - model.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), - &model.DispatchTableMessage{ + protocol.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.DispatchTableMessage{ OwnerRev: 1, Epoch: agent.CurrentEpoch(), ID: 1, @@ -284,7 +276,7 @@ func TestAgentBasics(t *testing.T) { Return(model.Ts(1000), model.Ts(1000)) require.Eventually(t, func() bool { - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) if len(suite.tableExecutor.Running) != 1 { return false @@ -293,7 +285,7 @@ func TestAgentBasics(t *testing.T) { case <-suite.ctx.Done(): require.Fail(t, "context should not be canceled") case msg := <-suite.checkpointCh: - require.Equal(t, &model.CheckpointMessage{ + require.Equal(t, &protocol.CheckpointMessage{ CheckpointTs: model.Ts(1000), ResolvedTs: model.Ts(1000), }, msg) @@ -309,7 +301,7 @@ func TestAgentBasics(t *testing.T) { suite.tableExecutor.On("GetCheckpoint").Return(model.Ts(1000), model.Ts(1000)) // Test Point 4: Accept an incoming DispatchTableMessage, and the AddTable method in TableExecutor can return true. - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) require.Eventually(t, func() bool { @@ -317,7 +309,7 @@ func TestAgentBasics(t *testing.T) { case <-suite.ctx.Done(): return false case msg := <-suite.dispatchResponseCh: - require.Equal(t, &model.DispatchTableResponseMessage{ + require.Equal(t, &protocol.DispatchTableResponseMessage{ ID: 1, Epoch: agent.CurrentEpoch(), }, msg) @@ -325,7 +317,7 @@ func TestAgentBasics(t *testing.T) { default: } - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) return false }, time.Second*3, time.Millisecond*10) @@ -354,27 +346,27 @@ func TestAgentNoOwnerAtStartUp(t *testing.T) { // Test Point 2: First ticks should not panic for i := 0; i < 10; i++ { - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) } // Test Point 3: Agent should process the Announce message. _, err = suite.ownerMessageClient.SendMessage(suite.ctx, - model.AnnounceTopic(model.DefaultChangeFeedID("cf-1")), - &model.AnnounceMessage{ + protocol.AnnounceTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.AnnounceMessage{ OwnerRev: 1, OwnerVersion: version.ReleaseSemver(), }) require.NoError(t, err) require.Eventually(t, func() bool { - err := agent.Tick(suite.cdcCtx) + err := agent.Tick(suite.ctx) require.NoError(t, err) select { case <-suite.ctx.Done(): require.Fail(t, "context should not be canceled") case syncMsg := <-suite.syncCh: - require.Equal(t, &model.SyncMessage{ + require.Equal(t, &protocol.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), Epoch: agent.CurrentEpoch(), Running: nil, @@ -423,7 +415,7 @@ func TestAgentTolerateClientClosed(t *testing.T) { // Test Point 2: We should tolerate the error ErrPeerMessageClientClosed for i := 0; i < 6; i++ { - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) } @@ -431,7 +423,7 @@ func TestAgentTolerateClientClosed(t *testing.T) { case <-suite.ctx.Done(): require.Fail(t, "context should not be canceled") case syncMsg := <-suite.syncCh: - require.Equal(t, &model.SyncMessage{ + require.Equal(t, &protocol.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), Epoch: agent.CurrentEpoch(), Running: nil, @@ -462,12 +454,12 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { suite.BlockSync() - err = agent.Tick(suite.cdcCtx) + err = agent.Tick(suite.ctx) require.NoError(t, err) _, err = suite.ownerMessageClient.SendMessage(suite.ctx, - model.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), - &model.DispatchTableMessage{ + protocol.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.DispatchTableMessage{ OwnerRev: 1, Epoch: agent.CurrentEpoch(), ID: 1, @@ -477,8 +469,8 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { require.NoError(t, err) _, err = suite.ownerMessageClient.SendMessage(suite.ctx, - model.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), - &model.DispatchTableMessage{ + protocol.DispatchTableTopic(model.DefaultChangeFeedID("cf-1")), + &protocol.DispatchTableMessage{ OwnerRev: 1, Epoch: agent.CurrentEpoch(), ID: 2, @@ -508,7 +500,7 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { start := time.Now() for time.Since(start) < 100*time.Millisecond { - err := agent.Tick(suite.cdcCtx) + err := agent.Tick(suite.ctx) require.NoError(t, err) select { @@ -530,7 +522,7 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { require.Fail(t, "context should not be canceled") return false case syncMsg := <-suite.syncCh: - require.Equal(t, &model.SyncMessage{ + require.Equal(t, &protocol.SyncMessage{ ProcessorVersion: version.ReleaseSemver(), Epoch: agent.CurrentEpoch(), Running: nil, @@ -541,7 +533,7 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { default: } - err := agent.Tick(suite.cdcCtx) + err := agent.Tick(suite.ctx) require.NoError(t, err) return false }, 1*time.Second, 10*time.Millisecond) @@ -555,7 +547,7 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { default: } - err := agent.Tick(suite.cdcCtx) + err := agent.Tick(suite.ctx) require.NoError(t, err) return false }, time.Second*3, time.Millisecond*10) @@ -569,7 +561,7 @@ func TestNoFinishOperationBeforeSyncIsReceived(t *testing.T) { default: } - err := agent.Tick(suite.cdcCtx) + err := agent.Tick(suite.ctx) require.NoError(t, err) return false }, time.Second*3, time.Millisecond*10) diff --git a/cdc/model/protocol.go b/cdc/scheduler/internal/base/protocol/protocol.go similarity index 83% rename from cdc/model/protocol.go rename to cdc/scheduler/internal/base/protocol/protocol.go index dc63752eae5..ea750f5dbb1 100644 --- a/cdc/model/protocol.go +++ b/cdc/scheduler/internal/base/protocol/protocol.go @@ -11,12 +11,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -package model +package protocol import ( "fmt" "github.com/pingcap/errors" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/p2p" "github.com/vmihailenco/msgpack/v5" ) @@ -25,7 +26,7 @@ import ( // FIXME a detailed documentation on the interaction will be added later in a separate file. // DispatchTableTopic returns a message topic for dispatching a table. -func DispatchTableTopic(changefeedID ChangeFeedID) p2p.Topic { +func DispatchTableTopic(changefeedID model.ChangeFeedID) p2p.Topic { return fmt.Sprintf("dispatch/%s/%s", changefeedID.Namespace, changefeedID.ID) } @@ -33,25 +34,25 @@ func DispatchTableTopic(changefeedID ChangeFeedID) p2p.Topic { type DispatchTableMessage struct { OwnerRev int64 `json:"owner-rev"` Epoch ProcessorEpoch `json:"epoch"` - ID TableID `json:"id"` - StartTs Ts `json:"start-ts"` + ID model.TableID `json:"id"` + StartTs model.Ts `json:"start-ts"` IsDelete bool `json:"is-delete"` } // DispatchTableResponseTopic returns a message topic for the result of // dispatching a table. It is sent from the Processor to the Owner. -func DispatchTableResponseTopic(changefeedID ChangeFeedID) p2p.Topic { +func DispatchTableResponseTopic(changefeedID model.ChangeFeedID) p2p.Topic { return fmt.Sprintf("dispatch-resp/%s/%s", changefeedID.Namespace, changefeedID.ID) } // DispatchTableResponseMessage is the message body for the result of dispatching a table. type DispatchTableResponseMessage struct { - ID TableID `json:"id"` + ID model.TableID `json:"id"` Epoch ProcessorEpoch `json:"epoch"` } // AnnounceTopic returns a message topic for announcing an ownership change. -func AnnounceTopic(changefeedID ChangeFeedID) p2p.Topic { +func AnnounceTopic(changefeedID model.ChangeFeedID) p2p.Topic { return fmt.Sprintf("send-status/%s/%s", changefeedID.Namespace, changefeedID.ID) } @@ -63,7 +64,7 @@ type AnnounceMessage struct { } // SyncTopic returns a message body for syncing the current states of a processor. -func SyncTopic(changefeedID ChangeFeedID) p2p.Topic { +func SyncTopic(changefeedID model.ChangeFeedID) p2p.Topic { return fmt.Sprintf("send-status-resp/%s/%s", changefeedID.Namespace, changefeedID.ID) } @@ -81,9 +82,9 @@ type SyncMessage struct { // it has to re-sync its states with the Owner. Epoch ProcessorEpoch - Running []TableID - Adding []TableID - Removing []TableID + Running []model.TableID + Adding []model.TableID + Removing []model.TableID } // Marshal serializes the message into MsgPack format. @@ -102,13 +103,13 @@ func (m *SyncMessage) Unmarshal(data []byte) error { // CheckpointTopic returns a topic for sending the latest checkpoint from // the Processor to the Owner. -func CheckpointTopic(changefeedID ChangeFeedID) p2p.Topic { +func CheckpointTopic(changefeedID model.ChangeFeedID) p2p.Topic { return fmt.Sprintf("checkpoint/%s/%s", changefeedID.Namespace, changefeedID.ID) } // CheckpointMessage is the message body for sending the latest checkpoint // from the Processor to the Owner. type CheckpointMessage struct { - CheckpointTs Ts `json:"checkpoint-ts"` - ResolvedTs Ts `json:"resolved-ts"` + CheckpointTs model.Ts `json:"checkpoint-ts"` + ResolvedTs model.Ts `json:"resolved-ts"` } diff --git a/cdc/model/protocol_test.go b/cdc/scheduler/internal/base/protocol/protocol_test.go similarity index 91% rename from cdc/model/protocol_test.go rename to cdc/scheduler/internal/base/protocol/protocol_test.go index 5a4d8590620..72958f376d2 100644 --- a/cdc/model/protocol_test.go +++ b/cdc/scheduler/internal/base/protocol/protocol_test.go @@ -11,12 +11,13 @@ // See the License for the specific language governing permissions and // limitations under the License. -package model +package protocol import ( "encoding/json" "testing" + "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/pkg/p2p" "github.com/stretchr/testify/require" ) @@ -28,7 +29,7 @@ var _ p2p.Serializable = (*SyncMessage)(nil) // TestChangefeedNameCannotIncludeSlash asserts that changefeed names cannot include slash. // Or otherwise the topic name encoding would be problematic. func TestChangefeedNameCannotIncludeSlash(t *testing.T) { - err := ValidateChangefeedID("a/b") + err := model.ValidateChangefeedID("a/b") require.Error(t, err, "changefeed name cannot include slash") } @@ -49,9 +50,9 @@ func TestSerializeSyncMessage(t *testing.T) { } func makeVeryLargeSyncMessage() *SyncMessage { - largeSliceFn := func() (ret []TableID) { + largeSliceFn := func() (ret []model.TableID) { for i := 0; i < 80000; i++ { - ret = append(ret, TableID(i)) + ret = append(ret, model.TableID(i)) } return } @@ -67,7 +68,7 @@ func TestMarshalDispatchTableMessage(t *testing.T) { OwnerRev: 1, StartTs: 2, Epoch: "test-epoch", - ID: TableID(1), + ID: model.TableID(1), IsDelete: true, } bytes, err := json.Marshal(msg) @@ -77,7 +78,7 @@ func TestMarshalDispatchTableMessage(t *testing.T) { func TestMarshalDispatchTableResponseMessage(t *testing.T) { msg := &DispatchTableResponseMessage{ - ID: TableID(1), + ID: model.TableID(1), Epoch: "test-epoch", } bytes, err := json.Marshal(msg) diff --git a/cdc/scheduler/schedule_dispatcher.go b/cdc/scheduler/internal/base/schedule_dispatcher.go similarity index 80% rename from cdc/scheduler/schedule_dispatcher.go rename to cdc/scheduler/internal/base/schedule_dispatcher.go index af711f5cb61..6b5149618b1 100644 --- a/cdc/scheduler/schedule_dispatcher.go +++ b/cdc/scheduler/internal/base/schedule_dispatcher.go @@ -11,49 +11,22 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "math" "sync" "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" - "github.com/pingcap/tiflow/pkg/context" + sched "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" "go.uber.org/zap" ) -const ( - // CheckpointCannotProceed is a placeholder indicating that the - // Owner should not advance the global checkpoint TS just yet. - CheckpointCannotProceed = model.Ts(0) -) - -// ScheduleDispatcher is an interface for a table scheduler used in Owner. -type ScheduleDispatcher interface { - // Tick is called periodically to update the SchedulerDispatcher on the latest state of replication. - // This function should NOT be assumed to be thread-safe. No concurrent calls allowed. - Tick( - ctx context.Context, - // Latest global checkpoint of the changefeed - checkpointTs model.Ts, - // All tables that SHOULD be replicated (or started) at the current checkpoint. - currentTables []model.TableID, - // All captures that are alive according to the latest Etcd states. - captures map[model.CaptureID]*model.CaptureInfo, - ) (newCheckpointTs, newResolvedTs model.Ts, err error) - - // MoveTable requests that a table be moved to target. - // It should be thread-safe. - MoveTable(tableID model.TableID, target model.CaptureID) - - // Rebalance triggers a rebalance operation. - // It should be thread-safe - Rebalance() -} - // ScheduleDispatcherCommunicator is an interface for the BaseScheduleDispatcher to // send commands to Processors. The owner of a BaseScheduleDispatcher should provide // an implementation of ScheduleDispatcherCommunicator to supply BaseScheduleDispatcher @@ -66,7 +39,7 @@ type ScheduleDispatcherCommunicator interface { startTs model.Ts, captureID model.CaptureID, isDelete bool, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, ) (done bool, err error) // Announce announces to the specified capture that the current node has become the Owner. @@ -80,15 +53,19 @@ const ( captureCountUninitialized = -1 ) -// BaseScheduleDispatcher implements the basic logic of a ScheduleDispatcher. +// ScheduleDispatcher implements the basic logic of a ScheduleDispatcher. // For it to be directly useful to the Owner, the Owner should implement it own // ScheduleDispatcherCommunicator. -type BaseScheduleDispatcher struct { - mu sync.Mutex - tables *util.TableSet // information of all actually running tables - captures map[model.CaptureID]*model.CaptureInfo // basic information of all captures - captureStatus map[model.CaptureID]*captureStatus // more information on the captures - checkpointTs model.Ts // current checkpoint-ts +type ScheduleDispatcher struct { + mu sync.Mutex + // information of all actually running tables + tables *util.TableSet + // basic information of all captures + captures map[model.CaptureID]*model.CaptureInfo + // more information on the captures + captureStatus map[model.CaptureID]*captureStatus + // current checkpoint-ts + checkpointTs model.Ts moveTableManager moveTableManager balancer balancer @@ -107,13 +84,13 @@ func NewBaseScheduleDispatcher( changeFeedID model.ChangeFeedID, communicator ScheduleDispatcherCommunicator, checkpointTs model.Ts, -) *BaseScheduleDispatcher { +) *ScheduleDispatcher { // logger is just the global logger with the `changefeed-id` field attached. logger := log.L().With( zap.String("namespace", changeFeedID.Namespace), zap.String("changefeed", changeFeedID.ID)) - return &BaseScheduleDispatcher{ + return &ScheduleDispatcher{ tables: util.NewTableSet(), captureStatus: map[model.CaptureID]*captureStatus{}, moveTableManager: newMoveTableManager(), @@ -134,7 +111,7 @@ type captureStatus struct { // Epoch is reset when the processor's internal states // have been reset. - Epoch model.ProcessorEpoch + Epoch protocol.ProcessorEpoch // Watermark fields CheckpointTs model.Ts @@ -157,7 +134,7 @@ const ( ) // Tick implements the interface ScheduleDispatcher. -func (s *BaseScheduleDispatcher) Tick( +func (s *ScheduleDispatcher) Tick( ctx context.Context, checkpointTs model.Ts, // currentTables are tables that SHOULD be running given the current checkpoint-ts. @@ -173,7 +150,7 @@ func (s *BaseScheduleDispatcher) Tick( s.captures = captures // We trigger an automatic rebalance if the capture count has changed. - // This logic is the same as in the older implementation of scheduler. + // This logic is the same as in the older implementation of sched. // TODO a better criterion is needed. // NOTE: We need to check whether the capture count has changed in every tick, // and set needRebalance to true if it has. If we miss a capture count change, @@ -197,13 +174,13 @@ func (s *BaseScheduleDispatcher) Tick( // Makes sure that captures have all been synchronized before proceeding. done, err := s.syncCaptures(ctx) if err != nil { - return CheckpointCannotProceed, CheckpointCannotProceed, errors.Trace(err) + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, errors.Trace(err) } if !done { // Returns early if not all captures have synced their states with us. // We need to know all captures' status in order to proceed. // This is crucial for ensuring that no table is double-scheduled. - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } s.descheduleTablesFromDownCaptures() @@ -223,10 +200,10 @@ func (s *BaseScheduleDispatcher) Tick( for _, tableID := range toAdd { ok, err := s.addTable(ctx, tableID, checkpointTs) if err != nil { - return CheckpointCannotProceed, CheckpointCannotProceed, errors.Trace(err) + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, errors.Trace(err) } if !ok { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } } @@ -242,10 +219,10 @@ func (s *BaseScheduleDispatcher) Tick( ok, err := s.removeTable(ctx, tableID) if err != nil { - return CheckpointCannotProceed, CheckpointCannotProceed, errors.Trace(err) + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, errors.Trace(err) } if !ok { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } } @@ -255,40 +232,40 @@ func (s *BaseScheduleDispatcher) Tick( s.tables.CountTableByStatus(util.RemovingTable) == 0 } if !checkAllTasksNormal() { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } // handleMoveTableJobs tries to execute user-specified manual move table jobs. ok, err := s.handleMoveTableJobs(ctx) if err != nil { - return CheckpointCannotProceed, CheckpointCannotProceed, errors.Trace(err) + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, errors.Trace(err) } if !ok { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } if !checkAllTasksNormal() { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } if s.needRebalance { ok, err := s.rebalance(ctx, checkpointTs) if err != nil { - return CheckpointCannotProceed, CheckpointCannotProceed, errors.Trace(err) + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, errors.Trace(err) } if !ok { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } s.needRebalance = false } if !checkAllTasksNormal() { - return CheckpointCannotProceed, CheckpointCannotProceed, nil + return sched.CheckpointCannotProceed, sched.CheckpointCannotProceed, nil } newCheckpointTs, resolvedTs = s.calculateTs() return } -func (s *BaseScheduleDispatcher) calculateTs() (checkpointTs, resolvedTs model.Ts) { +func (s *ScheduleDispatcher) calculateTs() (checkpointTs, resolvedTs model.Ts) { checkpointTs = math.MaxUint64 resolvedTs = math.MaxUint64 @@ -312,7 +289,7 @@ func (s *BaseScheduleDispatcher) calculateTs() (checkpointTs, resolvedTs model.T return } -func (s *BaseScheduleDispatcher) syncCaptures(ctx context.Context) (capturesAllSynced bool, err error) { +func (s *ScheduleDispatcher) syncCaptures(ctx context.Context) (capturesAllSynced bool, err error) { for captureID := range s.captureStatus { if _, ok := s.captures[captureID]; !ok { // removes expired captures from the captureSynced map @@ -359,7 +336,7 @@ func (s *BaseScheduleDispatcher) syncCaptures(ctx context.Context) (capturesAllS // descheduleTablesFromDownCaptures removes tables from `s.tables` that are // associated with a capture that no longer exists. // `s.captures` MUST be updated before calling this method. -func (s *BaseScheduleDispatcher) descheduleTablesFromDownCaptures() { +func (s *ScheduleDispatcher) descheduleTablesFromDownCaptures() { for _, captureID := range s.tables.GetDistinctCaptures() { // If the capture is not in the current list of captures, it means that // the capture has been removed from the system. @@ -375,7 +352,7 @@ func (s *BaseScheduleDispatcher) descheduleTablesFromDownCaptures() { } } -func (s *BaseScheduleDispatcher) findDiffTables( +func (s *ScheduleDispatcher) findDiffTables( shouldReplicateTables map[model.TableID]struct{}, ) (toAdd, toRemove []model.TableID) { // Find tables that need to be added. @@ -396,7 +373,7 @@ func (s *BaseScheduleDispatcher) findDiffTables( return } -func (s *BaseScheduleDispatcher) addTable( +func (s *ScheduleDispatcher) addTable( ctx context.Context, tableID model.TableID, startTs model.Ts, @@ -436,7 +413,7 @@ func (s *BaseScheduleDispatcher) addTable( return true, nil } -func (s *BaseScheduleDispatcher) removeTable( +func (s *ScheduleDispatcher) removeTable( ctx context.Context, tableID model.TableID, ) (done bool, err error) { @@ -462,7 +439,7 @@ func (s *BaseScheduleDispatcher) removeTable( } // MoveTable implements the interface SchedulerDispatcher. -func (s *BaseScheduleDispatcher) MoveTable(tableID model.TableID, target model.CaptureID) { +func (s *ScheduleDispatcher) MoveTable(tableID model.TableID, target model.CaptureID) { if !s.moveTableManager.Add(tableID, target) { log.Info("Move Table command has been ignored, because the last user triggered"+ "move has not finished", @@ -471,9 +448,11 @@ func (s *BaseScheduleDispatcher) MoveTable(tableID model.TableID, target model.C } } -func (s *BaseScheduleDispatcher) handleMoveTableJobs(ctx context.Context) (bool, error) { +func (s *ScheduleDispatcher) handleMoveTableJobs(ctx context.Context) (bool, error) { removeAllDone, err := s.moveTableManager.DoRemove(ctx, - func(ctx context.Context, tableID model.TableID, target model.CaptureID) (removeTableResult, error) { + func( + ctx context.Context, tableID model.TableID, target model.CaptureID, + ) (removeTableResult, error) { _, ok := s.tables.GetTableRecord(tableID) if !ok { s.logger.Warn("table does not exist", zap.Int64("tableID", tableID)) @@ -504,12 +483,12 @@ func (s *BaseScheduleDispatcher) handleMoveTableJobs(ctx context.Context) (bool, } // Rebalance implements the interface ScheduleDispatcher. -func (s *BaseScheduleDispatcher) Rebalance() { +func (s *ScheduleDispatcher) Rebalance() { s.needRebalance = true } -func (s *BaseScheduleDispatcher) rebalance(ctx context.Context, - checkpointTs model.Ts, +func (s *ScheduleDispatcher) rebalance( + ctx context.Context, checkpointTs model.Ts, ) (done bool, err error) { tablesToRemove := s.balancer.FindVictims(s.tables, s.captures) for _, record := range tablesToRemove { @@ -537,10 +516,10 @@ func (s *BaseScheduleDispatcher) rebalance(ctx context.Context, // OnAgentFinishedTableOperation is called when a table operation has been finished by // the processor. -func (s *BaseScheduleDispatcher) OnAgentFinishedTableOperation( +func (s *ScheduleDispatcher) OnAgentFinishedTableOperation( captureID model.CaptureID, tableID model.TableID, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, ) { s.mu.Lock() defer s.mu.Unlock() @@ -594,9 +573,9 @@ func (s *BaseScheduleDispatcher) OnAgentFinishedTableOperation( } // OnAgentSyncTaskStatuses is called when the processor sends its complete current state. -func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses( +func (s *ScheduleDispatcher) OnAgentSyncTaskStatuses( captureID model.CaptureID, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, running, adding, removing []model.TableID, ) { s.mu.Lock() @@ -632,7 +611,9 @@ func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses( zap.Int64("tableID", tableID), zap.String("actualCaptureID", record.CaptureID)) } - s.tables.AddTableRecord(&util.TableRecord{TableID: tableID, CaptureID: captureID, Status: util.AddingTable}) + s.tables.AddTableRecord(&util.TableRecord{ + TableID: tableID, CaptureID: captureID, Status: util.AddingTable, + }) } for _, tableID := range running { if record, ok := s.tables.GetTableRecord(tableID); ok { @@ -640,7 +621,9 @@ func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses( zap.Int64("tableID", tableID), zap.String("actualCaptureID", record.CaptureID)) } - s.tables.AddTableRecord(&util.TableRecord{TableID: tableID, CaptureID: captureID, Status: util.RunningTable}) + s.tables.AddTableRecord(&util.TableRecord{ + TableID: tableID, CaptureID: captureID, Status: util.RunningTable, + }) } for _, tableID := range removing { if record, ok := s.tables.GetTableRecord(tableID); ok { @@ -648,7 +631,9 @@ func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses( zap.Int64("tableID", tableID), zap.String("actualCaptureID", record.CaptureID)) } - s.tables.AddTableRecord(&util.TableRecord{TableID: tableID, CaptureID: captureID, Status: util.RemovingTable}) + s.tables.AddTableRecord(&util.TableRecord{ + TableID: tableID, CaptureID: captureID, Status: util.RemovingTable, + }) } status := s.captureStatus[captureID] @@ -657,7 +642,9 @@ func (s *BaseScheduleDispatcher) OnAgentSyncTaskStatuses( } // OnAgentCheckpoint is called when the processor sends a checkpoint. -func (s *BaseScheduleDispatcher) OnAgentCheckpoint(captureID model.CaptureID, checkpointTs model.Ts, resolvedTs model.Ts) { +func (s *ScheduleDispatcher) OnAgentCheckpoint( + captureID model.CaptureID, checkpointTs model.Ts, resolvedTs model.Ts, +) { s.mu.Lock() defer s.mu.Unlock() diff --git a/cdc/scheduler/schedule_dispatcher_test.go b/cdc/scheduler/internal/base/schedule_dispatcher_test.go similarity index 84% rename from cdc/scheduler/schedule_dispatcher_test.go rename to cdc/scheduler/internal/base/schedule_dispatcher_test.go index bc775664df7..fff367f87c7 100644 --- a/cdc/scheduler/schedule_dispatcher_test.go +++ b/cdc/scheduler/internal/base/schedule_dispatcher_test.go @@ -11,16 +11,18 @@ // See the License for the specific language governing permissions and // limitations under the License. -package scheduler +package base import ( + "context" "fmt" "testing" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" - "github.com/pingcap/tiflow/cdc/scheduler/util" - cdcContext "github.com/pingcap/tiflow/pkg/context" + "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base/protocol" + "github.com/pingcap/tiflow/cdc/scheduler/internal/util" "github.com/stretchr/testify/mock" "github.com/stretchr/testify/require" "go.uber.org/zap" @@ -56,13 +58,13 @@ func (m *mockScheduleDispatcherCommunicator) Reset() { } func (m *mockScheduleDispatcherCommunicator) DispatchTable( - ctx cdcContext.Context, + ctx context.Context, changeFeedID model.ChangeFeedID, tableID model.TableID, startTs model.Ts, captureID model.CaptureID, isDelete bool, - epoch model.ProcessorEpoch, + epoch protocol.ProcessorEpoch, ) (done bool, err error) { if !m.isBenchmark { log.Info("dispatch table called", @@ -82,7 +84,7 @@ func (m *mockScheduleDispatcherCommunicator) DispatchTable( } func (m *mockScheduleDispatcherCommunicator) Announce( - ctx cdcContext.Context, + ctx context.Context, changeFeedID model.ChangeFeedID, captureID model.CaptureID, ) (done bool, err error) { @@ -105,7 +107,8 @@ var defaultMockCaptureInfos = map[model.CaptureID]*model.CaptureInfo{ func TestDispatchTable(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -116,8 +119,8 @@ func TestDispatchTable(t *testing.T) { cf1, "capture-2").Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) @@ -130,8 +133,8 @@ func TestDispatchTable(t *testing.T) { Return(false, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) communicator.Reset() @@ -146,8 +149,8 @@ func TestDispatchTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) require.NotEqual(t, 0, len(communicator.addTableRecords["capture-1"])) require.NotEqual(t, 0, len(communicator.addTableRecords["capture-2"])) @@ -160,8 +163,8 @@ func TestDispatchTable(t *testing.T) { communicator.ExpectedCalls = nil checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) @@ -189,7 +192,8 @@ func TestDispatchTable(t *testing.T) { func TestSyncCaptures(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -201,8 +205,8 @@ func TestSyncCaptures(t *testing.T) { checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.Reset() communicator.On("Announce", mock.Anything, cf1, "capture-1"). @@ -211,14 +215,14 @@ func TestSyncCaptures(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{1, 2, 3}, []model.TableID{4, 5}, []model.TableID{6, 7}) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.Reset() dispatcher.OnAgentFinishedTableOperation("capture-1", 4, defaultEpoch) @@ -226,8 +230,8 @@ func TestSyncCaptures(t *testing.T) { dispatcher.OnAgentSyncTaskStatuses("capture-2", defaultEpoch, []model.TableID(nil), []model.TableID(nil), []model.TableID(nil)) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.Reset() dispatcher.OnAgentFinishedTableOperation("capture-1", 6, defaultEpoch) @@ -243,7 +247,8 @@ func TestSyncUnknownCapture(t *testing.T) { mockCaptureInfos := map[model.CaptureID]*model.CaptureInfo{} - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() dispatcher := NewBaseScheduleDispatcher( model.DefaultChangeFeedID("cf-1"), communicator, @@ -256,14 +261,15 @@ func TestSyncUnknownCapture(t *testing.T) { // We expect the `Sync` to be ignored. checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3, 4, 5}, mockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) } func TestRemoveTable(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -308,8 +314,8 @@ func TestRemoveTable(t *testing.T) { Return(false, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) communicator.Reset() @@ -318,8 +324,8 @@ func TestRemoveTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1500, []model.TableID{1, 2}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.OnAgentFinishedTableOperation("capture-1", 3, defaultEpoch) @@ -341,7 +347,8 @@ func TestCaptureGone(t *testing.T) { // capture-2 is gone } - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -380,15 +387,16 @@ func TestCaptureGone(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3}, mockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) } func TestCaptureRestarts(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -428,8 +436,8 @@ func TestCaptureRestarts(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1500, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) } @@ -447,7 +455,8 @@ func TestCaptureGoneWhileMovingTable(t *testing.T) { }, } - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -487,8 +496,8 @@ func TestCaptureGoneWhileMovingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, mockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) delete(mockCaptureInfos, "capture-2") @@ -502,8 +511,8 @@ func TestCaptureGoneWhileMovingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, mockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) } @@ -525,7 +534,8 @@ func TestRebalance(t *testing.T) { }, } - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -563,8 +573,8 @@ func TestRebalance(t *testing.T) { Return(false, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6}, mockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) communicator.AssertNumberOfCalls(t, "DispatchTable", 1) @@ -574,8 +584,8 @@ func TestRebalance(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6}, mockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertNumberOfCalls(t, "DispatchTable", 2) communicator.AssertExpectations(t) } @@ -598,7 +608,8 @@ func TestIgnoreEmptyCapture(t *testing.T) { }, } - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() dispatcher := NewBaseScheduleDispatcher( model.DefaultChangeFeedID("cf-1"), @@ -638,7 +649,8 @@ func TestIgnoreEmptyCapture(t *testing.T) { func TestIgnoreDeadCapture(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() dispatcher := NewBaseScheduleDispatcher( model.DefaultChangeFeedID("cf-1"), communicator, @@ -676,7 +688,8 @@ func TestIgnoreDeadCapture(t *testing.T) { func TestIgnoreUnsyncedCaptures(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() dispatcher := NewBaseScheduleDispatcher( model.DefaultChangeFeedID("cf-1"), @@ -707,8 +720,8 @@ func TestIgnoreUnsyncedCaptures(t *testing.T) { dispatcher.OnAgentCheckpoint("capture-2", 1000, 1000) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.Reset() dispatcher.OnAgentSyncTaskStatuses("capture-2", defaultEpoch, []model.TableID{2, 4, 6}, []model.TableID{}, []model.TableID{}) @@ -722,7 +735,8 @@ func TestIgnoreUnsyncedCaptures(t *testing.T) { func TestRebalanceWhileAddingTable(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -753,16 +767,16 @@ func TestRebalanceWhileAddingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6, 7}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.Rebalance() communicator.Reset() checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6, 7}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.OnAgentFinishedTableOperation("capture-2", model.TableID(7), defaultEpoch) @@ -772,8 +786,8 @@ func TestRebalanceWhileAddingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3, 4, 5, 6, 7}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertNumberOfCalls(t, "DispatchTable", 2) communicator.AssertExpectations(t) } @@ -781,7 +795,8 @@ func TestRebalanceWhileAddingTable(t *testing.T) { func TestManualMoveTableWhileAddingTable(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -815,14 +830,14 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) dispatcher.MoveTable(1, "capture-1") checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.OnAgentFinishedTableOperation("capture-2", 1, defaultEpoch) @@ -832,8 +847,8 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.OnAgentFinishedTableOperation("capture-2", 1, defaultEpoch) @@ -843,8 +858,8 @@ func TestManualMoveTableWhileAddingTable(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) } @@ -860,7 +875,8 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -882,8 +898,8 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { cf1, "capture-2").Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) dispatcher.OnAgentSyncTaskStatuses("capture-1", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) @@ -901,8 +917,8 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) require.NotEqual(t, 0, len(communicator.addTableRecords["capture-1"])) require.NotEqual(t, 0, len(communicator.addTableRecords["capture-2"])) @@ -915,8 +931,8 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { communicator.ExpectedCalls = nil checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) @@ -929,16 +945,16 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { cf1, "capture-3").Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) communicator.ExpectedCalls = nil dispatcher.OnAgentSyncTaskStatuses("capture-3", defaultEpoch, []model.TableID{}, []model.TableID{}, []model.TableID{}) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) for captureID, tables := range communicator.addTableRecords { @@ -956,8 +972,8 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { }) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) removedTableID := communicator.removeTableRecords[removeTableFromCapture][0] @@ -971,15 +987,16 @@ func TestAutoRebalanceOnCaptureOnline(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1000, []model.TableID{1, 2, 3}, captureList) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) communicator.AssertExpectations(t) } func TestInvalidFinishedTableOperation(t *testing.T) { t.Parallel() - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() cf1 := model.DefaultChangeFeedID("cf-1") dispatcher := NewBaseScheduleDispatcher(cf1, communicator, 1000) @@ -1013,15 +1030,15 @@ func TestInvalidFinishedTableOperation(t *testing.T) { Return(true, nil) checkpointTs, resolvedTs, err := dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) // Invalid epoch dispatcher.OnAgentFinishedTableOperation("capture-2", model.TableID(1), "invalid-epoch") checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) record, ok := dispatcher.tables.GetTableRecord(model.TableID(1)) require.True(t, ok) require.Equal(t, record.Status, util.AddingTable) @@ -1030,8 +1047,8 @@ func TestInvalidFinishedTableOperation(t *testing.T) { dispatcher.OnAgentFinishedTableOperation("capture-invalid", model.TableID(1), defaultEpoch) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) record, ok = dispatcher.tables.GetTableRecord(model.TableID(1)) require.True(t, ok) require.Equal(t, record.Status, util.AddingTable) @@ -1040,8 +1057,8 @@ func TestInvalidFinishedTableOperation(t *testing.T) { dispatcher.OnAgentFinishedTableOperation("capture-1", model.TableID(999), defaultEpoch) checkpointTs, resolvedTs, err = dispatcher.Tick(ctx, 1300, []model.TableID{1, 2, 3}, defaultMockCaptureInfos) require.NoError(t, err) - require.Equal(t, CheckpointCannotProceed, checkpointTs) - require.Equal(t, CheckpointCannotProceed, resolvedTs) + require.Equal(t, internal.CheckpointCannotProceed, checkpointTs) + require.Equal(t, internal.CheckpointCannotProceed, resolvedTs) record, ok = dispatcher.tables.GetTableRecord(model.TableID(1)) require.True(t, ok) require.Equal(t, record.Status, util.AddingTable) @@ -1053,7 +1070,8 @@ func TestInvalidFinishedTableOperation(t *testing.T) { } func BenchmarkAddTable(b *testing.B) { - ctx := cdcContext.NewBackendContext4Test(false) + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() communicator := NewMockScheduleDispatcherCommunicator() communicator.isBenchmark = true diff --git a/cdc/scheduler/internal/info_provider.go b/cdc/scheduler/internal/info_provider.go new file mode 100644 index 00000000000..1edf2bcb3a0 --- /dev/null +++ b/cdc/scheduler/internal/info_provider.go @@ -0,0 +1,36 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "github.com/pingcap/tiflow/cdc/model" +) + +// InfoProvider is the interface to get information about the internal states of the scheduler. +// We need this interface so that we can provide the information through HTTP API. +type InfoProvider interface { + // GetTaskStatuses returns the task statuses. + GetTaskStatuses() (map[model.CaptureID]*model.TaskStatus, error) + + // GetTaskPositions returns the task positions. + GetTaskPositions() (map[model.CaptureID]*model.TaskPosition, error) + + // GetTotalTableCounts returns the number of tables associated + // with each capture. + GetTotalTableCounts() map[model.CaptureID]int + + // GetPendingTableCounts returns the number of tables in a non-ready + // status (Adding & Removing) associated with each capture. + GetPendingTableCounts() map[model.CaptureID]int +} diff --git a/cdc/scheduler/internal/scheduler.go b/cdc/scheduler/internal/scheduler.go new file mode 100644 index 00000000000..4c440418ea9 --- /dev/null +++ b/cdc/scheduler/internal/scheduler.go @@ -0,0 +1,58 @@ +// Copyright 2021 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "context" + + "github.com/pingcap/tiflow/cdc/model" +) + +const ( + // CheckpointCannotProceed is a placeholder indicating that the + // Owner should not advance the global checkpoint TS just yet. + CheckpointCannotProceed = model.Ts(0) +) + +// Scheduler is an interface for scheduling tables. +// Since in our design, we do not record checkpoints per table, +// how we calculate the global watermarks (checkpoint-ts and resolved-ts) +// is heavily coupled with how tables are scheduled. +// That is why we have a scheduler interface that also reports the global watermarks. +type Scheduler interface { + // Tick is called periodically from the owner, and returns + // updated global watermarks. + // It is not thread-safe. + Tick( + ctx context.Context, + // Latest global checkpoint of the changefeed + checkpointTs model.Ts, + // All tables that SHOULD be replicated (or started) at the current checkpoint. + currentTables []model.TableID, + // All captures that are alive according to the latest Etcd states. + aliveCaptures map[model.CaptureID]*model.CaptureInfo, + ) (newCheckpointTs, newResolvedTs model.Ts, err error) + + // MoveTable requests that a table be moved to target. + // It is thread-safe. + MoveTable(tableID model.TableID, target model.CaptureID) + + // Rebalance triggers a rebalance operation. + // It is thread-safe + Rebalance() + + // Close scheduler and release it's resource. + // It is not thread-safe. + Close(ctx context.Context) +} diff --git a/cdc/scheduler/internal/table_executor.go b/cdc/scheduler/internal/table_executor.go new file mode 100644 index 00000000000..261b5b2df8a --- /dev/null +++ b/cdc/scheduler/internal/table_executor.go @@ -0,0 +1,46 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package internal + +import ( + "context" + + "github.com/pingcap/tiflow/cdc/model" +) + +// TableExecutor is an abstraction for "Processor". +// +// This interface is so designed that it would be the least problematic +// to adapt the current Processor implementation to it. +// TODO find a way to make the semantics easier to understand. +type TableExecutor interface { + AddTable(ctx context.Context, tableID model.TableID, startTs model.Ts) (done bool, err error) + RemoveTable(ctx context.Context, tableID model.TableID) (done bool, err error) + IsAddTableFinished(ctx context.Context, tableID model.TableID) (done bool) + IsRemoveTableFinished(ctx context.Context, tableID model.TableID) (done bool) + + // GetAllCurrentTables should return all tables that are being run, + // being added and being removed. + // + // NOTE: two subsequent calls to the method should return the same + // result, unless there is a call to AddTable, RemoveTable, IsAddTableFinished + // or IsRemoveTableFinished in between two calls to this method. + GetAllCurrentTables() []model.TableID + + // GetCheckpoint returns the local checkpoint-ts and resolved-ts of + // the processor. Its calculation should take into consideration all + // tables that would have been returned if GetAllCurrentTables had been + // called immediately before. + GetCheckpoint() (checkpointTs, resolvedTs model.Ts) +} diff --git a/cdc/scheduler/util/sort_table_ids.go b/cdc/scheduler/internal/util/sort_table_ids.go similarity index 100% rename from cdc/scheduler/util/sort_table_ids.go rename to cdc/scheduler/internal/util/sort_table_ids.go diff --git a/cdc/scheduler/util/table_set.go b/cdc/scheduler/internal/util/table_set.go similarity index 100% rename from cdc/scheduler/util/table_set.go rename to cdc/scheduler/internal/util/table_set.go diff --git a/cdc/scheduler/util/table_set_test.go b/cdc/scheduler/internal/util/table_set_test.go similarity index 100% rename from cdc/scheduler/util/table_set_test.go rename to cdc/scheduler/internal/util/table_set_test.go diff --git a/cdc/scheduler/rexport.go b/cdc/scheduler/rexport.go new file mode 100644 index 00000000000..6d4afb3b9ac --- /dev/null +++ b/cdc/scheduler/rexport.go @@ -0,0 +1,79 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package scheduler + +import ( + "context" + + "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/base" + "github.com/pingcap/tiflow/pkg/etcd" + "github.com/pingcap/tiflow/pkg/p2p" +) + +// TableExecutor is an abstraction for "Processor". +// +// This interface is so designed that it would be the least problematic +// to adapt the current Processor implementation to it. +// TODO find a way to make the semantics easier to understand. +type TableExecutor internal.TableExecutor + +// Scheduler is an interface for scheduling tables. +// Since in our design, we do not record checkpoints per table, +// how we calculate the global watermarks (checkpoint-ts and resolved-ts) +// is heavily coupled with how tables are scheduled. +// That is why we have a scheduler interface that also reports the global watermarks. +type Scheduler internal.Scheduler + +// InfoProvider is the interface to get information about the internal states of the scheduler. +// We need this interface so that we can provide the information through HTTP API. +type InfoProvider internal.InfoProvider + +// Agent is an interface for an object inside Processor that is responsible +// for receiving commands from the Owner. +// Ideally the processor should drive the Agent by Tick. +// +// Note that Agent is not thread-safe +type Agent internal.Agent + +// CheckpointCannotProceed is a placeholder indicating that the +// Owner should not advance the global checkpoint TS just yet. +const CheckpointCannotProceed = internal.CheckpointCannotProceed + +// NewAgent returns processor agent. +func NewAgent( + ctx context.Context, + messageServer *p2p.MessageServer, + messageRouter p2p.MessageRouter, + etcdClient *etcd.CDCEtcdClient, + executor TableExecutor, + changefeedID model.ChangeFeedID, +) (Agent, error) { + return base.NewAgent( + ctx, messageServer, messageRouter, etcdClient, executor, changefeedID) +} + +// NewScheduler returns owner scheduler. +func NewScheduler( + ctx context.Context, + changeFeedID model.ChangeFeedID, + checkpointTs model.Ts, + messageServer *p2p.MessageServer, + messageRouter p2p.MessageRouter, + ownerRevision int64, +) (Scheduler, error) { + return base.NewSchedulerV2( + ctx, changeFeedID, checkpointTs, messageServer, messageRouter, ownerRevision) +} From 7b7695bb14867fc34295f618026d5b55cf31354f Mon Sep 17 00:00:00 2001 From: Chunzhu Li Date: Fri, 13 May 2022 16:36:36 +0800 Subject: [PATCH 3/3] syncer: fix dm skip mariadb event problem (#5401) close pingcap/tiflow#5394 --- dm/syncer/syncer.go | 49 ++++++++++++++++++++----- dm/tests/gtid/data/db1.increment.sql | 3 +- dm/tests/gtid/data/db1.increment2.sql | 2 ++ dm/tests/gtid/data/db2.increment.sql | 3 +- dm/tests/gtid/data/db2.increment2.sql | 2 ++ dm/tests/gtid/run.sh | 51 +++++++++++++++++++++++++++ 6 files changed, 100 insertions(+), 10 deletions(-) create mode 100644 dm/tests/gtid/data/db1.increment2.sql create mode 100644 dm/tests/gtid/data/db2.increment2.sql diff --git a/dm/syncer/syncer.go b/dm/syncer/syncer.go index 1bcaf80fcc1..0db696b65a7 100644 --- a/dm/syncer/syncer.go +++ b/dm/syncer/syncer.go @@ -40,6 +40,7 @@ import ( "github.com/pingcap/tidb/util/filter" regexprrouter "github.com/pingcap/tidb/util/regexpr-router" router "github.com/pingcap/tidb/util/table-router" + "github.com/pingcap/tiflow/dm/pkg/gtid" clientv3 "go.etcd.io/etcd/client/v3" "go.uber.org/atomic" "go.uber.org/zap" @@ -1699,7 +1700,7 @@ func (s *Syncer) Run(ctx context.Context) (err error) { // startLocation is the start location for current received event // currentLocation is the end location for current received event (End_log_pos in `show binlog events` for mysql) - // lastLocation is the end location for last received (ROTATE / QUERY / XID) event + // lastLocation is the end location for last received and fully executed (ROTATE / QUERY / XID) event // we use startLocation to replace and skip binlog event of specified position // we use currentLocation and update table checkpoint in sharding ddl // we use lastLocation to update global checkpoint and table checkpoint @@ -1866,6 +1867,31 @@ func (s *Syncer) Run(ctx context.Context) (err error) { return nil } + advanceLatestLocationGtidSet := func(e *replication.BinlogEvent) error { + if _, ok := e.Event.(*replication.MariadbGTIDEvent); ok { + gtidSet, err2 := gtid.ParserGTID(s.cfg.Flavor, currentGTID) + if err2 != nil { + return err2 + } + if currentLocation.GetGTID().Contain(gtidSet) { + return nil + } + } + + // clone currentLocation's gtid set to avoid its gtid is transport to table checkpoint + // currently table checkpoint will save location's gtid set with shallow copy + newGTID := currentLocation.GetGTID().Clone() + err2 := newGTID.Update(currentGTID) + if err2 != nil { + return terror.Annotatef(err2, "fail to update GTID %s", currentGTID) + } + err2 = currentLocation.SetGTID(newGTID.Origin()) + if err2 != nil { + return terror.Annotatef(err2, "fail to set GTID %s", newGTID.Origin()) + } + return nil + } + // eventIndex is the rows event index in this transaction, it's used to avoiding read duplicate event in gtid mode eventIndex := 0 // affectedSourceTables is used for gtid mode to update table point's gtid set after receiving a xid event, @@ -2080,10 +2106,11 @@ func (s *Syncer) Run(ctx context.Context) (err error) { Name: lastLocation.Position.Name, Pos: e.Header.LogPos, }, - lastLocation.GetGTID(), + currentLocation.GetGTID(), ) currentLocation.Suffix = endSuffix + // TODO: can be removed in the future if queryEvent, ok := ev.(*replication.QueryEvent); ok { err = currentLocation.SetGTID(queryEvent.GSet) if err != nil { @@ -2242,11 +2269,6 @@ func (s *Syncer) Run(ctx context.Context) (err error) { } currentLocation.Position.Pos = e.Header.LogPos - err = currentLocation.SetGTID(ev.GSet) - if err != nil { - return terror.Annotatef(err, "fail to record GTID %v", ev.GSet) - } - s.tctx.L().Debug("", zap.String("event", "XID"), zap.Stringer("last location", lastLocation), log.WrapStringerField("location", currentLocation)) lastLocation.Position.Pos = e.Header.LogPos // update lastPos err = lastLocation.SetGTID(ev.GSet) @@ -2275,6 +2297,13 @@ func (s *Syncer) Run(ctx context.Context) (err error) { if err2 != nil { return err2 } + if s.cfg.EnableGTID { + // TODO: add mariaDB integration test + err2 = advanceLatestLocationGtidSet(e) + if err2 != nil { + return err2 + } + } } if err2 != nil { if err := s.handleEventError(err2, startLocation, currentLocation, e.Header.EventType == replication.QUERY_EVENT, originSQL); err != nil { @@ -2429,7 +2458,7 @@ func (s *Syncer) handleRowsEvent(ev *replication.RowsEvent, ec eventContext) (*f Name: ec.lastLocation.Position.Name, Pos: ec.header.LogPos, }, - ec.lastLocation.GetGTID(), + ec.currentLocation.GetGTID(), ) if ec.shardingReSync != nil { @@ -2657,6 +2686,10 @@ func generateExtendColumn(data [][]interface{}, r *regexprrouter.RouteTable, tab func (s *Syncer) handleQueryEvent(ev *replication.QueryEvent, ec eventContext, originSQL string) (err error) { if originSQL == "BEGIN" { + failpoint.Inject("NotUpdateLatestGTID", func(_ failpoint.Value) { + // directly return nil without update latest GTID here + failpoint.Return(nil) + }) // GTID event: GTID_NEXT = xxx:11 // Query event: BEGIN (GTID set = xxx:1-11) // Rows event: ... (GTID set = xxx:1-11) if we update lastLocation below, diff --git a/dm/tests/gtid/data/db1.increment.sql b/dm/tests/gtid/data/db1.increment.sql index 69a3f9740c4..38359e495d9 100644 --- a/dm/tests/gtid/data/db1.increment.sql +++ b/dm/tests/gtid/data/db1.increment.sql @@ -1,2 +1,3 @@ use gtid; -insert into t1 values (2); +alter table t1 add column b int; +insert into t1 values (2,2); diff --git a/dm/tests/gtid/data/db1.increment2.sql b/dm/tests/gtid/data/db1.increment2.sql new file mode 100644 index 00000000000..62f0366f717 --- /dev/null +++ b/dm/tests/gtid/data/db1.increment2.sql @@ -0,0 +1,2 @@ +use gtid; +insert into t1 values (3,3); diff --git a/dm/tests/gtid/data/db2.increment.sql b/dm/tests/gtid/data/db2.increment.sql index e2415de8222..52590c6a84c 100644 --- a/dm/tests/gtid/data/db2.increment.sql +++ b/dm/tests/gtid/data/db2.increment.sql @@ -1,2 +1,3 @@ use gtid; -insert into t2 values (2); \ No newline at end of file +alter table t2 add column b int; +insert into t2 values (2,2); diff --git a/dm/tests/gtid/data/db2.increment2.sql b/dm/tests/gtid/data/db2.increment2.sql new file mode 100644 index 00000000000..b3fbabdc9b9 --- /dev/null +++ b/dm/tests/gtid/data/db2.increment2.sql @@ -0,0 +1,2 @@ +use gtid; +insert into t2 values (3,3); diff --git a/dm/tests/gtid/run.sh b/dm/tests/gtid/run.sh index adca64f0320..a53cb040604 100755 --- a/dm/tests/gtid/run.sh +++ b/dm/tests/gtid/run.sh @@ -8,7 +8,58 @@ WORK_DIR=$TEST_DIR/$TEST_NAME API_VERSION="v1alpha1" TASK_NAME="test" +function advance_gtid_test() { + export GO_FAILPOINTS="github.com/pingcap/tiflow/dm/syncer/NotUpdateLatestGTID=return()" + run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + cp $cur/conf/source1.yaml $WORK_DIR/source1.yaml + cp $cur/conf/source2.yaml $WORK_DIR/source2.yaml + sed -i "/from:/i\relay-dir: $WORK_DIR/worker1/relay_log" $WORK_DIR/source1.yaml + sed -i "/from:/i\relay-dir: $WORK_DIR/worker2/relay_log" $WORK_DIR/source2.yaml + + # start DM worker and source one-by-one, make sure the source1 bound to worker1 + run_dm_master $WORK_DIR/master $MASTER_PORT $cur/conf/dm-master.toml + check_rpc_alive $cur/../bin/check_master_online 127.0.0.1:$MASTER_PORT + check_metric $MASTER_PORT 'start_leader_counter' 3 0 2 + run_dm_worker $WORK_DIR/worker1 $WORKER1_PORT $cur/conf/dm-worker1.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER1_PORT + dmctl_operate_source create $WORK_DIR/source1.yaml $SOURCE_ID1 + + run_dm_worker $WORK_DIR/worker2 $WORKER2_PORT $cur/conf/dm-worker2.toml + check_rpc_alive $cur/../bin/check_worker_online 127.0.0.1:$WORKER2_PORT + dmctl_operate_source create $WORK_DIR/source2.yaml $SOURCE_ID2 + + dmctl_start_task "$cur/conf/dm-task.yaml" "--remove-meta" + check_metric $WORKER1_PORT "dm_worker_task_state{source_id=\"mysql-replica-01\",task=\"test\",worker=\"worker1\"}" 10 1 3 + check_metric $WORKER2_PORT "dm_worker_task_state{source_id=\"mysql-replica-02\",task=\"test\",worker=\"worker2\"}" 10 1 3 + + run_sql_file $cur/data/db1.increment.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $cur/data/db2.increment.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "pause-task test" \ + "\"result\": true" 3 + + run_sql_file $cur/data/db1.increment2.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 + run_sql_file $cur/data/db2.increment2.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2 + + run_dm_ctl $WORK_DIR "127.0.0.1:$MASTER_PORT" \ + "resume-task test" \ + "\"result\": true" 3 + check_metric $WORKER1_PORT "dm_worker_task_state{source_id=\"mysql-replica-01\",task=\"test\",worker=\"worker1\"}" 10 1 3 + check_metric $WORKER2_PORT "dm_worker_task_state{source_id=\"mysql-replica-02\",task=\"test\",worker=\"worker2\"}" 10 1 3 + + check_sync_diff $WORK_DIR $cur/conf/diff_config.toml + + export GO_FAILPOINTS="" + cleanup_process $* + cleanup_data gtid +} + function run() { + advance_gtid_test + run_sql_file $cur/data/db1.prepare.sql $MYSQL_HOST1 $MYSQL_PORT1 $MYSQL_PASSWORD1 run_sql_file $cur/data/db2.prepare.sql $MYSQL_HOST2 $MYSQL_PORT2 $MYSQL_PASSWORD2