From 8c0fd4d3f72ec84451379bc42e0b93a1f8a09721 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Mon, 5 Dec 2022 17:13:32 +0800 Subject: [PATCH 1/5] scheduler: replace tableID with Span Signed-off-by: Neil Shen --- cdc/owner/changefeed.go | 15 +- cdc/owner/changefeed_test.go | 3 +- cdc/owner/owner_test.go | 5 +- cdc/scheduler/internal/v3/coordinator.go | 40 +- .../internal/v3/coordinator_bench_test.go | 11 +- cdc/scheduler/internal/v3/coordinator_test.go | 168 +++++--- cdc/scheduler/internal/v3/info_provider.go | 2 +- .../internal/v3/info_provider_test.go | 8 +- .../internal/v3/member/capture_manager.go | 14 +- .../v3/member/capture_manager_test.go | 63 +-- .../v3/replication/replication_manager.go | 276 +++++++------ .../replication/replication_manager_test.go | 379 +++++++++++------- .../v3/replication/replication_set.go | 58 +-- .../v3/replication/replication_set_test.go | 277 +++++++------ .../internal/v3/scheduler/scheduler.go | 7 +- .../v3/scheduler/scheduler_balance.go | 22 +- .../v3/scheduler/scheduler_balance_test.go | 27 +- .../internal/v3/scheduler/scheduler_basic.go | 71 ++-- .../v3/scheduler/scheduler_basic_test.go | 88 ++-- .../v3/scheduler/scheduler_drain_capture.go | 30 +- .../scheduler/scheduler_drain_capture_test.go | 44 +- .../v3/scheduler/scheduler_manager.go | 18 +- .../v3/scheduler/scheduler_manager_test.go | 37 +- .../v3/scheduler/scheduler_move_table.go | 79 ++-- .../v3/scheduler/scheduler_move_table_test.go | 34 +- .../v3/scheduler/scheduler_rebalance.go | 56 +-- .../v3/scheduler/scheduler_rebalance_test.go | 25 +- cdc/scheduler/rexport.go | 6 +- 28 files changed, 1071 insertions(+), 792 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index af81600e0b0..aa15323c933 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -42,7 +42,7 @@ import ( // newSchedulerFromCtx creates a new scheduler from context. // This function is factored out to facilitate unit testing. func newSchedulerFromCtx( - ctx cdcContext.Context, pdClock pdutil.Clock, + ctx cdcContext.Context, up *upstream.Upstream, ) (ret scheduler.Scheduler, err error) { changeFeedID := ctx.ChangefeedVars().ID messageServer := ctx.GlobalVars().MessageServer @@ -52,15 +52,14 @@ func newSchedulerFromCtx( cfg := config.GetGlobalServerConfig().Debug ret, err = scheduler.NewScheduler( ctx, captureID, changeFeedID, - messageServer, messageRouter, ownerRev, cfg.Scheduler, pdClock) + messageServer, messageRouter, ownerRev, up.RegionCache, up.PDClock, cfg.Scheduler) return ret, errors.Trace(err) } func newScheduler( - ctx cdcContext.Context, - pdClock pdutil.Clock, + ctx cdcContext.Context, up *upstream.Upstream, ) (scheduler.Scheduler, error) { - return newSchedulerFromCtx(ctx, pdClock) + return newSchedulerFromCtx(ctx, up) } type changefeed struct { @@ -127,7 +126,7 @@ type changefeed struct { ) (puller.DDLPuller, error) newSink func(changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, reportErr func(error)) DDLSink - newScheduler func(ctx cdcContext.Context, pdClock pdutil.Clock) (scheduler.Scheduler, error) + newScheduler func(ctx cdcContext.Context, up *upstream.Upstream) (scheduler.Scheduler, error) lastDDLTs uint64 // Timestamp of the last executed DDL. Only used for tests. } @@ -165,7 +164,7 @@ func newChangefeed4Test( changefeed model.ChangeFeedID, ) (puller.DDLPuller, error), newSink func(changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, reportErr func(err error)) DDLSink, - newScheduler func(ctx cdcContext.Context, pdClock pdutil.Clock) (scheduler.Scheduler, error), + newScheduler func(ctx cdcContext.Context, up *upstream.Upstream) (scheduler.Scheduler, error), ) *changefeed { c := newChangefeed(id, state, up) c.newDDLPuller = newDDLPuller @@ -540,7 +539,7 @@ LOOP: zap.String("changefeed", c.id.ID)) // create scheduler - c.scheduler, err = c.newScheduler(ctx, c.upstream.PDClock) + c.scheduler, err = c.newScheduler(ctx, c.upstream) if err != nil { return errors.Trace(err) } diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 2c96b23f604..391475837ef 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -35,7 +35,6 @@ import ( cdcContext "github.com/pingcap/tiflow/pkg/context" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/pdutil" "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/pingcap/tiflow/pkg/upstream" "github.com/stretchr/testify/require" @@ -216,7 +215,7 @@ func createChangefeed4Test(ctx cdcContext.Context, t *testing.T, }, // new scheduler func( - ctx cdcContext.Context, pdClock pdutil.Clock, + ctx cdcContext.Context, up *upstream.Upstream, ) (scheduler.Scheduler, error) { return &mockScheduler{}, nil }) diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index 0d74afd27c5..02cb0eb7d98 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -32,7 +32,6 @@ import ( cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" "github.com/pingcap/tiflow/pkg/orchestrator" - "github.com/pingcap/tiflow/pkg/pdutil" "github.com/pingcap/tiflow/pkg/txnutil/gc" "github.com/pingcap/tiflow/pkg/upstream" "github.com/pingcap/tiflow/pkg/version" @@ -63,7 +62,7 @@ func newOwner4Test( changefeed model.ChangeFeedID, ) (puller.DDLPuller, error), newSink func(changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, reportErr func(err error)) DDLSink, - newScheduler func(ctx cdcContext.Context, pdClock pdutil.Clock) (scheduler.Scheduler, error), + newScheduler func(ctx cdcContext.Context, up *upstream.Upstream) (scheduler.Scheduler, error), pdClient pd.Client, ) Owner { m := upstream.NewManager4Test(pdClient) @@ -103,7 +102,7 @@ func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*ownerImpl, *orches }, // new scheduler func( - ctx cdcContext.Context, pdClock pdutil.Clock, + ctx cdcContext.Context, up *upstream.Upstream, ) (scheduler.Scheduler, error) { return &mockScheduler{}, nil }, diff --git a/cdc/scheduler/internal/v3/coordinator.go b/cdc/scheduler/internal/v3/coordinator.go index 764154471d9..f4334e695aa 100644 --- a/cdc/scheduler/internal/v3/coordinator.go +++ b/cdc/scheduler/internal/v3/coordinator.go @@ -21,7 +21,10 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/compat" + "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/keyspan" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/scheduler" @@ -54,6 +57,8 @@ type coordinator struct { replicationM *replication.Manager captureM *member.CaptureManager schedulerM *scheduler.Manager + reconciler *keyspan.Reconciler + compat *compat.Compat pdClock pdutil.Clock lastCollectTime time.Time @@ -68,8 +73,9 @@ func NewCoordinator( messageServer *p2p.MessageServer, messageRouter p2p.MessageRouter, ownerRevision int64, - cfg *config.SchedulerConfig, + regionCache keyspan.RegionCache, pdClock pdutil.Clock, + cfg *config.SchedulerConfig, ) (internal.Scheduler, error) { trans, err := transport.NewTransport( ctx, changefeedID, transport.SchedulerRole, messageServer, messageRouter) @@ -78,6 +84,7 @@ func NewCoordinator( } coord := newCoordinator(captureID, changefeedID, ownerRevision, cfg) coord.trans = trans + coord.reconciler = keyspan.NewReconciler(changefeedID, regionCache, cfg.RegionPerSpan) coord.pdClock = pdClock return coord, nil } @@ -100,6 +107,7 @@ func newCoordinator( captureID, changefeedID, revision, cfg.HeartbeatTick), schedulerM: scheduler.NewSchedulerManager(changefeedID, cfg), changefeedID: changefeedID, + compat: compat.New(cfg, map[model.CaptureID]*model.CaptureInfo{}), } } @@ -120,6 +128,7 @@ func (c *coordinator) Tick( } // MoveTable implement the scheduler interface +// FIXME: tableID should be Span. func (c *coordinator) MoveTable(tableID model.TableID, target model.CaptureID) { c.mu.Lock() defer c.mu.Unlock() @@ -134,7 +143,7 @@ func (c *coordinator) MoveTable(tableID model.TableID, target model.CaptureID) { return } - c.schedulerM.MoveTable(tableID, target) + c.schedulerM.MoveTable(tablepb.Span{TableID: tableID}, target) } // Rebalance implement the scheduler interface @@ -171,11 +180,13 @@ func (c *coordinator) DrainCapture(target model.CaptureID) (int, error) { } var count int - for _, rep := range c.replicationM.ReplicationSets() { - if rep.Primary == target { - count++ - } - } + c.replicationM.ReplicationSets().Ascend( + func(_ tablepb.Span, rep *replication.ReplicationSet) bool { + if rep.Primary == target { + count++ + } + return true + }) if count == 0 { log.Info("schedulerv3: drain capture request ignored, "+ @@ -239,12 +250,15 @@ func (c *coordinator) Close(ctx context.Context) { // =========== func (c *coordinator) poll( - ctx context.Context, - checkpointTs model.Ts, - currentTables []model.TableID, + ctx context.Context, checkpointTs model.Ts, currentTables []model.TableID, aliveCaptures map[model.CaptureID]*model.CaptureInfo, ) (newCheckpointTs, newResolvedTs model.Ts, err error) { c.maybeCollectMetrics() + if c.compat.UpdateCaptureInfo(aliveCaptures) { + log.Info("schedulerv3: compat update capture info", + zap.Any("captures", aliveCaptures), + zap.Bool("spanReplicationEnabled", c.compat.CheckSpanReplicationEnabled())) + } recvMsgs, err := c.recvMsgs(ctx) if err != nil { @@ -294,8 +308,9 @@ func (c *coordinator) poll( // Generate schedule tasks based on the current status. replications := c.replicationM.ReplicationSets() runningTasks := c.replicationM.RunningTasks() + currentSpans := c.reconciler.Reconcile(ctx, currentTables, replications, c.compat) allTasks := c.schedulerM.Schedule( - checkpointTs, currentTables, c.captureM.Captures, replications, runningTasks) + checkpointTs, currentSpans, c.captureM.Captures, replications, runningTasks) // Handle generated schedule tasks. msgs, err = c.replicationM.HandleTasks(allTasks) @@ -329,6 +344,7 @@ func (c *coordinator) recvMsgs(ctx context.Context) ([]*schedulepb.Message, erro n++ } } + c.compat.AfterTransportReceive(recvMsgs[:n]) return recvMsgs[:n], nil } @@ -353,8 +369,8 @@ func (c *coordinator) sendMsgs(ctx context.Context, msgs []*schedulepb.Message) ProcessorEpoch: epoch, } m.From = c.captureID - } + c.compat.BeforeTransportSend(msgs) return c.trans.Send(ctx, msgs) } diff --git a/cdc/scheduler/internal/v3/coordinator_bench_test.go b/cdc/scheduler/internal/v3/coordinator_bench_test.go index 08cb4d99824..fc105003163 100644 --- a/cdc/scheduler/internal/v3/coordinator_bench_test.go +++ b/cdc/scheduler/internal/v3/coordinator_bench_test.go @@ -140,11 +140,12 @@ func BenchmarkCoordinatorHeartbeatResponse(b *testing.B) { tableID := int64(10000 + i) currentTables = append(currentTables, tableID) captureID := fmt.Sprint(i % captureCount) + span := tablepb.Span{TableID: tableID} rep, err := replication.NewReplicationSet( - tableID, 0, map[string]*tablepb.TableStatus{ + span, 0, map[string]*tablepb.TableStatus{ captureID: { - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }, }, model.ChangeFeedID{}) if err != nil { @@ -163,8 +164,8 @@ func BenchmarkCoordinatorHeartbeatResponse(b *testing.B) { heartbeatResp[captureID].HeartbeatResponse.Tables = append( heartbeatResp[captureID].HeartbeatResponse.Tables, tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }) } recvMsgs := make([]*schedulepb.Message, 0, len(heartbeatResp)) diff --git a/cdc/scheduler/internal/v3/coordinator_test.go b/cdc/scheduler/internal/v3/coordinator_test.go index 9dfb0a7c50d..1fcf287b211 100644 --- a/cdc/scheduler/internal/v3/coordinator_test.go +++ b/cdc/scheduler/internal/v3/coordinator_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" + "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/keyspan" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/scheduler" @@ -28,6 +29,7 @@ import ( "github.com/pingcap/tiflow/pkg/config" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/leakutil" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -38,13 +40,12 @@ func TestMain(m *testing.M) { func TestCoordinatorSendMsgs(t *testing.T) { t.Parallel() ctx := context.Background() - trans := transport.NewMockTrans() - coord := coordinator{ - version: "6.2.0", - revision: schedulepb.OwnerRevision{Revision: 3}, - captureID: "0", - trans: trans, - } + coord, trans := newTestCoordinator(&config.SchedulerConfig{ + RegionPerSpan: 10000, // Enable span replication. + }) + coord.version = "6.2.0" + coord.revision = schedulepb.OwnerRevision{Revision: 3} + coord.captureID = "0" coord.captureM = member.NewCaptureManager("", model.ChangeFeedID{}, coord.revision, 0) coord.sendMsgs( ctx, []*schedulepb.Message{{To: "1", MsgType: schedulepb.MsgDispatchTableRequest}}) @@ -75,13 +76,12 @@ func TestCoordinatorRecvMsgs(t *testing.T) { t.Parallel() ctx := context.Background() - trans := transport.NewMockTrans() - coord := coordinator{ - version: "6.2.0", - revision: schedulepb.OwnerRevision{Revision: 3}, - captureID: "0", - trans: trans, - } + coord, trans := newTestCoordinator(&config.SchedulerConfig{ + RegionPerSpan: 10000, // Enable span replication. + }) + coord.version = "6.2.0" + coord.revision = schedulepb.OwnerRevision{Revision: 3} + coord.captureID = "0" trans.RecvBuffer = append(trans.RecvBuffer, &schedulepb.Message{ @@ -115,16 +115,96 @@ func TestCoordinatorRecvMsgs(t *testing.T) { }}, msgs) } +func TestCoordinatorTransportCompat(t *testing.T) { + t.Parallel() + + coord, trans := newTestCoordinator(&config.SchedulerConfig{ + RegionPerSpan: 0, // Disable span replication. + }) + + ctx := context.Background() + // Test compat.BeforeTransportSend. + coord.sendMsgs( + ctx, []*schedulepb.Message{{ + To: "b", + MsgType: schedulepb.MsgDispatchTableRequest, + DispatchTableRequest: &schedulepb.DispatchTableRequest{ + Request: &schedulepb.DispatchTableRequest_AddTable{ + AddTable: &schedulepb.AddTableRequest{Span: spanz.TableIDToComparableSpan(1)}, + }, + }, + }}) + + require.EqualValues(t, []*schedulepb.Message{{ + Header: &schedulepb.Message_Header{ + Version: coord.version, + OwnerRevision: coord.revision, + }, + From: "a", To: "b", MsgType: schedulepb.MsgDispatchTableRequest, + DispatchTableRequest: &schedulepb.DispatchTableRequest{ + Request: &schedulepb.DispatchTableRequest_AddTable{ + AddTable: &schedulepb.AddTableRequest{ + TableID: 1, + Span: spanz.TableIDToComparableSpan(1), + }, + }, + }, + }}, trans.SendBuffer) + + // Test compat.AfterTransportReceive. + trans.RecvBuffer = append(trans.RecvBuffer, + &schedulepb.Message{ + Header: &schedulepb.Message_Header{ + OwnerRevision: coord.revision, + }, + From: "b", To: coord.captureID, MsgType: schedulepb.MsgDispatchTableResponse, + DispatchTableResponse: &schedulepb.DispatchTableResponse{ + Response: &schedulepb.DispatchTableResponse_AddTable{ + AddTable: &schedulepb.AddTableResponse{ + Status: &tablepb.TableStatus{ + TableID: 1, + }, + }, + }, + }, + }) + msgs, err := coord.recvMsgs(ctx) + require.NoError(t, err) + require.EqualValues(t, []*schedulepb.Message{{ + Header: &schedulepb.Message_Header{ + OwnerRevision: coord.revision, + }, + From: "b", To: coord.captureID, MsgType: schedulepb.MsgDispatchTableResponse, + DispatchTableResponse: &schedulepb.DispatchTableResponse{ + Response: &schedulepb.DispatchTableResponse_AddTable{ + AddTable: &schedulepb.AddTableResponse{ + Status: &tablepb.TableStatus{ + TableID: 1, + Span: spanz.TableIDToComparableSpan(1), + }, + }, + }, + }, + }}, msgs) +} + +func newTestCoordinator(cfg *config.SchedulerConfig) (*coordinator, *transport.MockTrans) { + coord := newCoordinator("a", model.ChangeFeedID{}, 1, cfg) + trans := transport.NewMockTrans() + coord.trans = trans + coord.reconciler = keyspan.NewReconciler( + model.ChangeFeedID{}, keyspan.NewMockRegionCache(), cfg.RegionPerSpan) + return coord, trans +} + func TestCoordinatorHeartbeat(t *testing.T) { t.Parallel() - coord := newCoordinator("a", model.ChangeFeedID{}, 1, &config.SchedulerConfig{ + coord, trans := newTestCoordinator(&config.SchedulerConfig{ HeartbeatTick: math.MaxInt, MaxTaskConcurrency: 1, AddTableBatchSize: 50, }) - trans := transport.NewMockTrans() - coord.trans = trans // Prepare captureM and replicationM. // Two captures "a", "b". @@ -158,8 +238,8 @@ func TestCoordinatorHeartbeat(t *testing.T) { MsgType: schedulepb.MsgHeartbeatResponse, HeartbeatResponse: &schedulepb.HeartbeatResponse{ Tables: []tablepb.TableStatus{ - {TableID: 1, State: tablepb.TableStateReplicating}, - {TableID: 2, State: tablepb.TableStateReplicating}, + {Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating}, + {Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStateReplicating}, }, }, }) @@ -170,18 +250,16 @@ func TestCoordinatorHeartbeat(t *testing.T) { msgs = trans.SendBuffer require.Len(t, msgs, 1) // Basic scheduler, make sure all tables get replicated. - require.EqualValues(t, 3, msgs[0].DispatchTableRequest.GetAddTable().TableID) - require.Len(t, coord.replicationM.GetReplicationSetForTests(), 3) + require.EqualValues(t, 3, msgs[0].DispatchTableRequest.GetAddTable().Span.TableID) + require.Equal(t, coord.replicationM.GetReplicationSetForTests().Len(), 3) } func TestCoordinatorAddCapture(t *testing.T) { t.Parallel() - coord := newCoordinator("a", model.ChangeFeedID{}, 1, &config.SchedulerConfig{ + coord, trans := newTestCoordinator(&config.SchedulerConfig{ HeartbeatTick: math.MaxInt, MaxTaskConcurrency: 1, }) - trans := transport.NewMockTrans() - coord.trans = trans // Prepare captureM and replicationM. // Two captures "a". @@ -191,15 +269,15 @@ func TestCoordinatorAddCapture(t *testing.T) { require.True(t, coord.captureM.CheckAllCaptureInitialized()) init := map[string][]tablepb.TableStatus{ "a": { - {TableID: 1, State: tablepb.TableStateReplicating}, - {TableID: 2, State: tablepb.TableStateReplicating}, - {TableID: 3, State: tablepb.TableStateReplicating}, + {Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating}, + {Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStateReplicating}, + {Span: spanz.TableIDToComparableSpan(3), State: tablepb.TableStateReplicating}, }, } msgs, err := coord.replicationM.HandleCaptureChanges(init, nil, 0) require.Nil(t, err) require.Len(t, msgs, 0) - require.Len(t, coord.replicationM.GetReplicationSetForTests(), 3) + require.Equal(t, coord.replicationM.GetReplicationSetForTests().Len(), 3) // Capture "b" is online, heartbeat, and then move one table to capture "b". ctx := context.Background() @@ -232,13 +310,11 @@ func TestCoordinatorAddCapture(t *testing.T) { func TestCoordinatorRemoveCapture(t *testing.T) { t.Parallel() - coord := newCoordinator("a", model.ChangeFeedID{}, 1, &config.SchedulerConfig{ + coord, trans := newTestCoordinator(&config.SchedulerConfig{ HeartbeatTick: math.MaxInt, MaxTaskConcurrency: 1, AddTableBatchSize: 50, }) - trans := transport.NewMockTrans() - coord.trans = trans // Prepare captureM and replicationM. // Three captures "a" "b" "c". @@ -249,14 +325,14 @@ func TestCoordinatorRemoveCapture(t *testing.T) { coord.captureM.SetInitializedForTests(true) require.True(t, coord.captureM.CheckAllCaptureInitialized()) init := map[string][]tablepb.TableStatus{ - "a": {{TableID: 1, State: tablepb.TableStateReplicating}}, - "b": {{TableID: 2, State: tablepb.TableStateReplicating}}, - "c": {{TableID: 3, State: tablepb.TableStateReplicating}}, + "a": {{Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating}}, + "b": {{Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStateReplicating}}, + "c": {{Span: spanz.TableIDToComparableSpan(3), State: tablepb.TableStateReplicating}}, } msgs, err := coord.replicationM.HandleCaptureChanges(init, nil, 0) require.Nil(t, err) require.Len(t, msgs, 0) - require.Len(t, coord.replicationM.GetReplicationSetForTests(), 3) + require.Equal(t, coord.replicationM.GetReplicationSetForTests().Len(), 3) // Capture "c" is removed, add table 3 to another capture. ctx := context.Background() @@ -267,7 +343,7 @@ func TestCoordinatorRemoveCapture(t *testing.T) { msgs = trans.SendBuffer require.Len(t, msgs, 1) require.NotNil(t, msgs[0].DispatchTableRequest.GetAddTable(), msgs[0]) - require.EqualValues(t, 3, msgs[0].DispatchTableRequest.GetAddTable().TableID) + require.EqualValues(t, 3, msgs[0].DispatchTableRequest.GetAddTable().Span.TableID) } func TestCoordinatorDrainCapture(t *testing.T) { @@ -293,7 +369,7 @@ func TestCoordinatorDrainCapture(t *testing.T) { require.Equal(t, 0, count) coord.replicationM.SetReplicationSetForTests(&replication.ReplicationSet{ - TableID: 1, + Span: spanz.TableIDToComparableSpan(1), State: replication.ReplicationSetStateReplicating, Primary: "a", }) @@ -304,7 +380,7 @@ func TestCoordinatorDrainCapture(t *testing.T) { coord.captureM.Captures["b"] = &member.CaptureStatus{State: member.CaptureStateInitialized} coord.replicationM.SetReplicationSetForTests(&replication.ReplicationSet{ - TableID: 2, + Span: spanz.TableIDToComparableSpan(2), State: replication.ReplicationSetStateReplicating, Primary: "b", }) @@ -323,12 +399,10 @@ func TestCoordinatorDrainCapture(t *testing.T) { func TestCoordinatorAdvanceCheckpoint(t *testing.T) { t.Parallel() - coord := newCoordinator("a", model.ChangeFeedID{}, 1, &config.SchedulerConfig{ + coord, trans := newTestCoordinator(&config.SchedulerConfig{ HeartbeatTick: math.MaxInt, MaxTaskConcurrency: 1, }) - trans := transport.NewMockTrans() - coord.trans = trans // Prepare captureM and replicationM. // Two captures "a", "b". @@ -359,13 +433,15 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { HeartbeatResponse: &schedulepb.HeartbeatResponse{ Tables: []tablepb.TableStatus{ { - TableID: 1, State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(1), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 2, ResolvedTs: 4, }, }, { - TableID: 2, State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(2), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 2, ResolvedTs: 4, }, @@ -392,13 +468,15 @@ func TestCoordinatorAdvanceCheckpoint(t *testing.T) { HeartbeatResponse: &schedulepb.HeartbeatResponse{ Tables: []tablepb.TableStatus{ { - TableID: 1, State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(1), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 3, ResolvedTs: 5, }, }, { - TableID: 2, State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(2), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 4, ResolvedTs: 5, }, diff --git a/cdc/scheduler/internal/v3/info_provider.go b/cdc/scheduler/internal/v3/info_provider.go index 75d6c06373d..eb331ea516c 100644 --- a/cdc/scheduler/internal/v3/info_provider.go +++ b/cdc/scheduler/internal/v3/info_provider.go @@ -40,7 +40,7 @@ func (c *coordinator) GetTaskStatuses() (map[model.CaptureID]*model.TaskStatus, Tables: make(map[model.TableID]*model.TableReplicaInfo), } for _, s := range status.Tables { - taskStatus.Tables[s.TableID] = &model.TableReplicaInfo{ + taskStatus.Tables[s.Span.TableID] = &model.TableReplicaInfo{ StartTs: s.Checkpoint.CheckpointTs, } } diff --git a/cdc/scheduler/internal/v3/info_provider_test.go b/cdc/scheduler/internal/v3/info_provider_test.go index 6d39b9eeb40..1adaca237ca 100644 --- a/cdc/scheduler/internal/v3/info_provider_test.go +++ b/cdc/scheduler/internal/v3/info_provider_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal" + "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/keyspan" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/pkg/config" "github.com/stretchr/testify/require" @@ -32,12 +33,15 @@ func TestInfoProvider(t *testing.T) { HeartbeatTick: math.MaxInt, MaxTaskConcurrency: 1, }) + cfg := config.NewDefaultSchedulerConfig() + coord.reconciler = keyspan.NewReconciler( + model.ChangeFeedID{}, keyspan.NewMockRegionCache(), cfg.RegionPerSpan) coord.captureM.Captures = map[model.CaptureID]*member.CaptureStatus{ "a": {Tables: []tablepb.TableStatus{{ - TableID: 1, + Span: tablepb.Span{TableID: 1}, Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, }, { - TableID: 2, + Span: tablepb.Span{TableID: 2}, Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, }}}, "b": {}, diff --git a/cdc/scheduler/internal/v3/member/capture_manager.go b/cdc/scheduler/internal/v3/member/capture_manager.go index 0ebdc92d2d8..8859a0008f1 100644 --- a/cdc/scheduler/internal/v3/member/capture_manager.go +++ b/cdc/scheduler/internal/v3/member/capture_manager.go @@ -19,6 +19,7 @@ import ( "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -163,26 +164,27 @@ func (c *CaptureManager) checkAllCaptureInitialized() bool { // Tick advances the logical lock of capture manager and produce heartbeat when // necessary. func (c *CaptureManager) Tick( - reps map[model.TableID]*replication.ReplicationSet, drainingCapture model.CaptureID, + reps *spanz.Map[*replication.ReplicationSet], drainingCapture model.CaptureID, ) []*schedulepb.Message { c.tickCounter++ if c.tickCounter < c.heartbeatTick { return nil } c.tickCounter = 0 - tables := make(map[model.CaptureID][]model.TableID) - for tableID, rep := range reps { + tables := make(map[model.CaptureID][]tablepb.Span) + reps.Ascend(func(span tablepb.Span, rep *replication.ReplicationSet) bool { for captureID := range rep.Captures { - tables[captureID] = append(tables[captureID], tableID) + tables[captureID] = append(tables[captureID], span) } - } + return true + }) msgs := make([]*schedulepb.Message, 0, len(c.Captures)) for to := range c.Captures { msgs = append(msgs, &schedulepb.Message{ To: to, MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{ - TableIDs: tables[to], + Spans: tables[to], // IsStopping let the receiver capture know that it should be stopping now. // At the moment, this is triggered by `DrainCapture` scheduler. IsStopping: drainingCapture == to, diff --git a/cdc/scheduler/internal/v3/member/capture_manager_test.go b/cdc/scheduler/internal/v3/member/capture_manager_test.go index 2dfd0406165..c0ee926402f 100644 --- a/cdc/scheduler/internal/v3/member/capture_manager_test.go +++ b/cdc/scheduler/internal/v3/member/capture_manager_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -92,13 +93,13 @@ func TestCaptureManagerHandleAliveCaptureUpdate(t *testing.T) { Header: &schedulepb.Message_Header{}, From: "2", MsgType: schedulepb.MsgHeartbeatResponse, HeartbeatResponse: &schedulepb.HeartbeatResponse{ - Tables: []tablepb.TableStatus{{TableID: 1}}, + Tables: []tablepb.TableStatus{{Span: tablepb.Span{TableID: 1}}}, }, }, { Header: &schedulepb.Message_Header{}, From: "3", MsgType: schedulepb.MsgHeartbeatResponse, HeartbeatResponse: &schedulepb.HeartbeatResponse{ - Tables: []tablepb.TableStatus{{TableID: 2}}, + Tables: []tablepb.TableStatus{{Span: tablepb.Span{TableID: 2}}}, }, }}) require.False(t, cm.CheckAllCaptureInitialized()) @@ -106,7 +107,10 @@ func TestCaptureManagerHandleAliveCaptureUpdate(t *testing.T) { require.Len(t, msgs, 0) require.True(t, cm.CheckAllCaptureInitialized()) require.EqualValues(t, &CaptureChanges{ - Init: map[string][]tablepb.TableStatus{"2": {{TableID: 1}}, "3": {{TableID: 2}}}, + Init: map[string][]tablepb.TableStatus{ + "2": {{Span: tablepb.Span{TableID: 1}}}, + "3": {{Span: tablepb.Span{TableID: 2}}}, + }, }, cm.TakeChanges()) // Add a new node and remove an old node. @@ -117,7 +121,7 @@ func TestCaptureManagerHandleAliveCaptureUpdate(t *testing.T) { {To: "4", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, }, msgs) require.Equal(t, &CaptureChanges{ - Removed: map[string][]tablepb.TableStatus{"2": {{TableID: 1}}}, + Removed: map[string][]tablepb.TableStatus{"2": {{Span: tablepb.Span{TableID: 1}}}}, }, cm.TakeChanges()) require.False(t, cm.CheckAllCaptureInitialized()) } @@ -181,9 +185,9 @@ func TestCaptureManagerTick(t *testing.T) { cm := NewCaptureManager("", model.ChangeFeedID{}, rev, 2) // No heartbeat if there is no capture. - msgs := cm.Tick(nil, captureIDNotDraining) + msgs := cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.Empty(t, msgs) - msgs = cm.Tick(nil, captureIDNotDraining) + msgs = cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.Empty(t, msgs) ms := map[model.CaptureID]*model.CaptureInfo{ @@ -193,9 +197,9 @@ func TestCaptureManagerTick(t *testing.T) { cm.HandleAliveCaptureUpdate(ms) // Heartbeat even if capture is uninitialized. - msgs = cm.Tick(nil, captureIDNotDraining) + msgs = cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.Empty(t, msgs) - msgs = cm.Tick(nil, captureIDNotDraining) + msgs = cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.ElementsMatch(t, []*schedulepb.Message{ {To: "1", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, {To: "2", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, @@ -205,9 +209,9 @@ func TestCaptureManagerTick(t *testing.T) { for _, s := range []CaptureState{CaptureStateInitialized, CaptureStateStopping} { cm.Captures["1"].State = s cm.Captures["2"].State = s - msgs = cm.Tick(nil, captureIDNotDraining) + msgs = cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.Empty(t, msgs) - msgs = cm.Tick(nil, captureIDNotDraining) + msgs = cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.ElementsMatch(t, []*schedulepb.Message{ {To: "1", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, {To: "2", MsgType: schedulepb.MsgHeartbeat, Heartbeat: &schedulepb.Heartbeat{}}, @@ -215,27 +219,38 @@ func TestCaptureManagerTick(t *testing.T) { } // TableID in heartbeat. - msgs = cm.Tick(nil, captureIDNotDraining) + msgs = cm.Tick(spanz.NewMap[*replication.ReplicationSet](), captureIDNotDraining) require.Empty(t, msgs) - tables := map[model.TableID]*replication.ReplicationSet{ - 1: {Captures: map[model.CaptureID]replication.Role{ + + tables := spanz.NewMap[*replication.ReplicationSet]() + tables.ReplaceOrInsert( + tablepb.Span{TableID: 1}, + &replication.ReplicationSet{Captures: map[model.CaptureID]replication.Role{ "1": replication.RolePrimary, - }}, - 2: {Captures: map[model.CaptureID]replication.Role{ + }}) + tables.ReplaceOrInsert( + tablepb.Span{TableID: 2}, + &replication.ReplicationSet{Captures: map[model.CaptureID]replication.Role{ "1": replication.RolePrimary, "2": replication.RoleSecondary, - }}, - 3: {Captures: map[model.CaptureID]replication.Role{ + }}) + tables.ReplaceOrInsert( + tablepb.Span{TableID: 3}, + &replication.ReplicationSet{Captures: map[model.CaptureID]replication.Role{ "2": replication.RoleSecondary, - }}, - 4: {}, - } + }}) + tables.ReplaceOrInsert(tablepb.Span{TableID: 4}, &replication.ReplicationSet{}) + msgs = cm.Tick(tables, captureIDNotDraining) require.Len(t, msgs, 2) if msgs[0].To == "1" { - require.ElementsMatch(t, []model.TableID{1, 2}, msgs[0].Heartbeat.TableIDs) - require.ElementsMatch(t, []model.TableID{2, 3}, msgs[1].Heartbeat.TableIDs) + require.ElementsMatch(t, + []tablepb.Span{{TableID: 1}, {TableID: 2}}, msgs[0].Heartbeat.Spans) + require.ElementsMatch(t, + []tablepb.Span{{TableID: 2}, {TableID: 3}}, msgs[1].Heartbeat.Spans) } else { - require.ElementsMatch(t, []model.TableID{2, 3}, msgs[0].Heartbeat.TableIDs) - require.ElementsMatch(t, []model.TableID{1, 2}, msgs[1].Heartbeat.TableIDs) + require.ElementsMatch(t, + []tablepb.Span{{TableID: 2}, {TableID: 3}}, msgs[0].Heartbeat.Spans) + require.ElementsMatch(t, + []tablepb.Span{{TableID: 1}, {TableID: 2}}, msgs[1].Heartbeat.Spans) } } diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index 900d5e8b415..ef3fb320035 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -14,6 +14,7 @@ package replication import ( + "bytes" "container/heap" "math" "time" @@ -24,6 +25,7 @@ import ( "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal" "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/tikv/client-go/v2/oracle" "go.uber.org/zap" ) @@ -49,20 +51,20 @@ type BurstBalance struct { // MoveTable is a schedule task for moving a table. type MoveTable struct { - TableID model.TableID + Span tablepb.Span DestCapture model.CaptureID } // AddTable is a schedule task for adding a table. type AddTable struct { - TableID model.TableID + Span tablepb.Span CaptureID model.CaptureID CheckpointTs model.Ts } // RemoveTable is a schedule task for removing a table. type RemoveTable struct { - TableID model.TableID + Span tablepb.Span CaptureID model.CaptureID } @@ -92,19 +94,19 @@ func (s *ScheduleTask) Name() string { // Manager manages replications and running scheduling tasks. type Manager struct { //nolint:revive - tables map[model.TableID]*ReplicationSet + spans *spanz.Map[*ReplicationSet] - runningTasks map[model.TableID]*ScheduleTask + runningTasks *spanz.Map[*ScheduleTask] maxTaskConcurrency int changefeedID model.ChangeFeedID - slowestTableID model.TableID - slowTableHeap SetHeap + slowestTableID tablepb.Span acceptAddTableTask int acceptRemoveTableTask int acceptMoveTableTask int acceptBurstBalanceTask int + slowTableHeap SetHeap lastLogSlowTablesTime time.Time } @@ -112,16 +114,11 @@ type Manager struct { //nolint:revive func NewReplicationManager( maxTaskConcurrency int, changefeedID model.ChangeFeedID, ) *Manager { - slowTableHeap := make(SetHeap, 0, defaultSlowTableHeapSize) - heap.Init(&slowTableHeap) - return &Manager{ - tables: make(map[int64]*ReplicationSet), - runningTasks: make(map[int64]*ScheduleTask), - maxTaskConcurrency: maxTaskConcurrency, - changefeedID: changefeedID, - slowTableHeap: slowTableHeap, - lastLogSlowTablesTime: time.Now(), + spans: spanz.NewMap[*ReplicationSet](), + runningTasks: spanz.NewMap[*ScheduleTask](), + maxTaskConcurrency: maxTaskConcurrency, + changefeedID: changefeedID, } } @@ -132,45 +129,57 @@ func (r *Manager) HandleCaptureChanges( checkpointTs model.Ts, ) ([]*schedulepb.Message, error) { if init != nil { - if len(r.tables) != 0 { + if r.spans.Len() != 0 { log.Panic("schedulerv3: init again", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), - zap.Any("init", init), zap.Any("tables", r.tables)) - } - tableStatus := map[model.TableID]map[model.CaptureID]*tablepb.TableStatus{} - for captureID, tables := range init { - for i := range tables { - table := tables[i] - if _, ok := tableStatus[table.TableID]; !ok { - tableStatus[table.TableID] = map[model.CaptureID]*tablepb.TableStatus{} + zap.Any("init", init), zap.Any("tablesCount", r.spans.Len())) + } + spanStatusMap := spanz.NewMap[map[model.CaptureID]*tablepb.TableStatus]() + for captureID, spans := range init { + for i := range spans { + table := spans[i] + if _, ok := spanStatusMap.Get(table.Span); !ok { + spanStatusMap.ReplaceOrInsert( + table.Span, map[model.CaptureID]*tablepb.TableStatus{}) } - tableStatus[table.TableID][captureID] = &table + spanStatusMap.GetV(table.Span)[captureID] = &table } } - for tableID, status := range tableStatus { - table, err := NewReplicationSet( - tableID, checkpointTs, status, r.changefeedID) + var err error + spanStatusMap.Ascend(func(span tablepb.Span, status map[string]*tablepb.TableStatus) bool { + table, err := NewReplicationSet(span, checkpointTs, status, r.changefeedID) if err != nil { - return nil, errors.Trace(err) + err = errors.Trace(err) + return false } - r.tables[tableID] = table + r.spans.ReplaceOrInsert(table.Span, table) + return true + }) + if err != nil { + return nil, errors.Trace(err) } } sentMsgs := make([]*schedulepb.Message, 0) if removed != nil { - for _, table := range r.tables { + var err error + r.spans.Ascend(func(span tablepb.Span, table *ReplicationSet) bool { for captureID := range removed { msgs, affected, err := table.handleCaptureShutdown(captureID) if err != nil { - return nil, errors.Trace(err) + err = errors.Trace(err) + return false } sentMsgs = append(sentMsgs, msgs...) if affected { // Cleanup its running task. - delete(r.runningTasks, table.TableID) + r.runningTasks.Delete(table.Span) } } + return true + }) + if err != nil { + return nil, errors.Trace(err) } } return sentMsgs, nil @@ -211,7 +220,7 @@ func (r *Manager) handleMessageHeartbeatResponse( ) ([]*schedulepb.Message, error) { sentMsgs := make([]*schedulepb.Message, 0) for _, status := range msg.Tables { - table, ok := r.tables[status.TableID] + table, ok := r.spans.Get(status.Span) if !ok { log.Info("schedulerv3: ignore table status no table found", zap.String("namespace", r.changefeedID.Namespace), @@ -227,8 +236,8 @@ func (r *Manager) handleMessageHeartbeatResponse( log.Info("schedulerv3: table has removed", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), - zap.Int64("tableID", status.TableID)) - delete(r.tables, status.TableID) + zap.Int64("tableID", status.Span.TableID)) + r.spans.Delete(status.Span) } sentMsgs = append(sentMsgs, msgs...) } @@ -252,7 +261,7 @@ func (r *Manager) handleMessageDispatchTableResponse( return nil, nil } - table, ok := r.tables[status.TableID] + table, ok := r.spans.Get(status.Span) if !ok { log.Info("schedulerv3: ignore table status no table found", zap.String("namespace", r.changefeedID.Namespace), @@ -268,8 +277,8 @@ func (r *Manager) handleMessageDispatchTableResponse( log.Info("schedulerv3: table has removed", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), - zap.Int64("tableID", status.TableID)) - delete(r.tables, status.TableID) + zap.Int64("tableID", status.Span.TableID)) + r.spans.Delete(status.Span) } return msgs, nil } @@ -279,17 +288,22 @@ func (r *Manager) HandleTasks( tasks []*ScheduleTask, ) ([]*schedulepb.Message, error) { // Check if a running task is finished. - for tableID := range r.runningTasks { - if table, ok := r.tables[tableID]; ok { + toBeDeleted := []tablepb.Span{} + r.runningTasks.Ascend(func(span tablepb.Span, task *ScheduleTask) bool { + if table, ok := r.spans.Get(span); ok { // If table is back to Replicating or Removed, // the running task is finished. if table.State == ReplicationSetStateReplicating || table.hasRemoved() { - delete(r.runningTasks, tableID) + toBeDeleted = append(toBeDeleted, span) } } else { // No table found, remove the task - delete(r.runningTasks, tableID) + toBeDeleted = append(toBeDeleted, span) } + return true + }) + for _, span := range toBeDeleted { + r.runningTasks.Delete(span) } sentMsgs := make([]*schedulepb.Message, 0) @@ -308,7 +322,7 @@ func (r *Manager) HandleTasks( } // Check if accepting one more task exceeds maxTaskConcurrency. - if len(r.runningTasks) == r.maxTaskConcurrency { + if r.runningTasks.Len() == r.maxTaskConcurrency { log.Debug("schedulerv3: too many running task", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID)) @@ -317,25 +331,25 @@ func (r *Manager) HandleTasks( continue } - var tableID model.TableID + var span tablepb.Span if task.AddTable != nil { - tableID = task.AddTable.TableID + span = task.AddTable.Span } else if task.RemoveTable != nil { - tableID = task.RemoveTable.TableID + span = task.RemoveTable.Span } else if task.MoveTable != nil { - tableID = task.MoveTable.TableID + span = task.MoveTable.Span } // Skip task if the table is already running a task, // or the table has removed. - if _, ok := r.runningTasks[tableID]; ok { + if _, ok := r.runningTasks.Get(span); ok { log.Info("schedulerv3: ignore task, already exists", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), zap.Any("task", task)) continue } - if _, ok := r.tables[tableID]; !ok && task.AddTable == nil { + if _, ok := r.spans.Get(span); !ok && task.AddTable == nil { log.Info("schedulerv3: ignore task, table not found", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), @@ -356,7 +370,7 @@ func (r *Manager) HandleTasks( return nil, errors.Trace(err) } sentMsgs = append(sentMsgs, msgs...) - r.runningTasks[tableID] = task + r.runningTasks.ReplaceOrInsert(span, task) if task.Accept != nil { task.Accept() } @@ -369,14 +383,13 @@ func (r *Manager) handleAddTableTask( ) ([]*schedulepb.Message, error) { r.acceptAddTableTask++ var err error - table := r.tables[task.TableID] - if table == nil { - table, err = NewReplicationSet( - task.TableID, task.CheckpointTs, nil, r.changefeedID) + table, ok := r.spans.Get(task.Span) + if !ok { + table, err = NewReplicationSet(task.Span, task.CheckpointTs, nil, r.changefeedID) if err != nil { return nil, errors.Trace(err) } - r.tables[task.TableID] = table + r.spans.ReplaceOrInsert(task.Span, table) } return table.handleAddTable(task.CaptureID) } @@ -385,13 +398,13 @@ func (r *Manager) handleRemoveTableTask( task *RemoveTable, ) ([]*schedulepb.Message, error) { r.acceptRemoveTableTask++ - table := r.tables[task.TableID] + table, _ := r.spans.Get(task.Span) if table.hasRemoved() { log.Info("schedulerv3: table has removed", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), - zap.Int64("tableID", task.TableID)) - delete(r.tables, task.TableID) + zap.Int64("tableID", task.Span.TableID)) + r.spans.Delete(task.Span) return nil, nil } return table.handleRemoveTable() @@ -401,7 +414,7 @@ func (r *Manager) handleMoveTableTask( task *MoveTable, ) ([]*schedulepb.Message, error) { r.acceptMoveTableTask++ - table := r.tables[task.TableID] + table, _ := r.spans.Get(task.Span) return table.handleMoveTable(task.DestCapture) } @@ -430,7 +443,7 @@ func (r *Manager) handleBurstBalanceTasks( sentMsgs := make([]*schedulepb.Message, 0, len(task.AddTables)) for i := range task.AddTables { addTable := task.AddTables[i] - if r.runningTasks[addTable.TableID] != nil { + if _, ok := r.runningTasks.Get(addTable.Span); ok { // Skip add table if the table is already running a task. continue } @@ -440,11 +453,11 @@ func (r *Manager) handleBurstBalanceTasks( } sentMsgs = append(sentMsgs, msgs...) // Just for place holding. - r.runningTasks[addTable.TableID] = &ScheduleTask{} + r.runningTasks.ReplaceOrInsert(addTable.Span, &ScheduleTask{}) } for i := range task.RemoveTables { removeTable := task.RemoveTables[i] - if r.runningTasks[removeTable.TableID] != nil { + if _, ok := r.runningTasks.Get(removeTable.Span); ok { // Skip add table if the table is already running a task. continue } @@ -454,11 +467,11 @@ func (r *Manager) handleBurstBalanceTasks( } sentMsgs = append(sentMsgs, msgs...) // Just for place holding. - r.runningTasks[removeTable.TableID] = &ScheduleTask{} + r.runningTasks.ReplaceOrInsert(removeTable.Span, &ScheduleTask{}) } for i := range task.MoveTables { moveTable := task.MoveTables[i] - if r.runningTasks[moveTable.TableID] != nil { + if _, ok := r.runningTasks.Get(moveTable.Span); ok { // Skip add table if the table is already running a task. continue } @@ -468,82 +481,102 @@ func (r *Manager) handleBurstBalanceTasks( } sentMsgs = append(sentMsgs, msgs...) // Just for place holding. - r.runningTasks[moveTable.TableID] = &ScheduleTask{} + r.runningTasks.ReplaceOrInsert(moveTable.Span, &ScheduleTask{}) } return sentMsgs, nil } // ReplicationSets return all tracking replication set // Caller must not modify the returned map. -func (r *Manager) ReplicationSets() map[model.TableID]*ReplicationSet { - return r.tables +func (r *Manager) ReplicationSets() *spanz.Map[*ReplicationSet] { + return r.spans } // RunningTasks return running tasks. // Caller must not modify the returned map. -func (r *Manager) RunningTasks() map[model.TableID]*ScheduleTask { +func (r *Manager) RunningTasks() *spanz.Map[*ScheduleTask] { return r.runningTasks } // AdvanceCheckpoint tries to advance checkpoint and returns current checkpoint. func (r *Manager) AdvanceCheckpoint( - currentTables []model.TableID, - currentTime time.Time, + currentTables []model.TableID, currentPDTime time.Time, ) (newCheckpointTs, newResolvedTs model.Ts) { newCheckpointTs, newResolvedTs = math.MaxUint64, math.MaxUint64 - slowestTableID := int64(0) + slowestRange := tablepb.Span{} for _, tableID := range currentTables { - table, ok := r.tables[tableID] - if !ok { + tableStart, tableEnd := spanz.TableIDToComparableRange(tableID) + tableSpanFound, tableHasHole := false, false + tableSpanStartFound, tableSpanEndFound := false, false + var lastSpan tablepb.Span + r.spans.AscendRange(tableStart, tableEnd, + func(span tablepb.Span, table *ReplicationSet) bool { + if lastSpan.TableID != 0 && !bytes.Equal(lastSpan.EndKey, span.StartKey) { + log.Warn("schedulerv3: span hole detected, skip advance checkpoint", + zap.String("namespace", r.changefeedID.Namespace), + zap.String("changefeed", r.changefeedID.ID), + zap.Stringer("lastSpan", &lastSpan), + zap.Stringer("span", &span)) + tableHasHole = true + return false + } + lastSpan = span + tableSpanFound = true + if bytes.Equal(span.StartKey, tableStart.StartKey) { + tableSpanStartFound = true + } + if bytes.Equal(span.EndKey, tableEnd.StartKey) { + tableSpanEndFound = true + } + + // Find the minimum checkpoint ts and resolved ts. + if newCheckpointTs > table.Checkpoint.CheckpointTs { + newCheckpointTs = table.Checkpoint.CheckpointTs + slowestRange = span + } + if newResolvedTs > table.Checkpoint.ResolvedTs { + newResolvedTs = table.Checkpoint.ResolvedTs + } + return true + }) + if !tableSpanFound || !tableSpanStartFound || !tableSpanEndFound || tableHasHole { // Can not advance checkpoint there is a table missing. - log.Warn("schedulerv3: cannot advance checkpoint since missing table", + log.Warn("schedulerv3: cannot advance checkpoint since missing span", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), zap.Int64("tableID", tableID)) return checkpointCannotProceed, checkpointCannotProceed } - // Find the minimum checkpoint ts and resolved ts. - if newCheckpointTs > table.Checkpoint.CheckpointTs { - newCheckpointTs = table.Checkpoint.CheckpointTs - slowestTableID = tableID - } - if newResolvedTs > table.Checkpoint.ResolvedTs { - newResolvedTs = table.Checkpoint.ResolvedTs - } } - if slowestTableID != 0 { - r.slowestTableID = slowestTableID + if slowestRange.TableID != 0 { + r.slowestTableID = slowestRange } // If changefeed's checkpoint lag is larger than 30s, // log the 4 slowlest table infos every minute, which can // help us find the problematic tables. - checkpointLag := currentTime.Sub(oracle.GetTimeFromTS(newCheckpointTs)) + checkpointLag := currentPDTime.Sub(oracle.GetTimeFromTS(newCheckpointTs)) if checkpointLag > logSlowTablesLagThreshold && time.Since(r.lastLogSlowTablesTime) > logSlowTablesInterval { - r.logSlowTableInfo(currentTables, currentTime) + r.logSlowTableInfo(currentPDTime) r.lastLogSlowTablesTime = time.Now() } return newCheckpointTs, newResolvedTs } -func (r *Manager) logSlowTableInfo(currentTables []model.TableID, currentTime time.Time) { +func (r *Manager) logSlowTableInfo(currentPDTime time.Time) { // find the slow tables - for _, tableID := range currentTables { - table, ok := r.tables[tableID] - if !ok { - continue - } - lag := currentTime.Sub(oracle.GetTimeFromTS(table.Checkpoint.CheckpointTs)) - if lag < logSlowTablesLagThreshold { - continue - } - heap.Push(&r.slowTableHeap, table) - if r.slowTableHeap.Len() > defaultSlowTableHeapSize { - heap.Pop(&r.slowTableHeap) + r.spans.Ascend(func(span tablepb.Span, table *ReplicationSet) bool { + lag := currentPDTime.Sub(oracle.GetTimeFromTS(table.Checkpoint.CheckpointTs)) + if lag > logSlowTablesLagThreshold { + heap.Push(&r.slowTableHeap, table) + if r.slowTableHeap.Len() > defaultSlowTableHeapSize { + heap.Pop(&r.slowTableHeap) + } } - } + return true + }) num := r.slowTableHeap.Len() for i := 0; i < num; i++ { @@ -551,11 +584,11 @@ func (r *Manager) logSlowTableInfo(currentTables []model.TableID, currentTime ti log.Info("schedulerv3: slow table", zap.String("namespace", r.changefeedID.Namespace), zap.String("changefeed", r.changefeedID.ID), - zap.Int64("tableID", table.TableID), + zap.Int64("tableID", table.Span.TableID), zap.String("tableStatus", table.Stats.String()), zap.Uint64("checkpointTs", table.Checkpoint.CheckpointTs), zap.Uint64("resolvedTs", table.Checkpoint.ResolvedTs), - zap.Duration("checkpointLag", currentTime. + zap.Duration("checkpointLag", currentPDTime. Sub(oracle.GetTimeFromTS(table.Checkpoint.CheckpointTs)))) } } @@ -564,10 +597,10 @@ func (r *Manager) logSlowTableInfo(currentTables []model.TableID, currentTime ti func (r *Manager) CollectMetrics() { cf := r.changefeedID tableGauge. - WithLabelValues(cf.Namespace, cf.ID).Set(float64(len(r.tables))) - if table, ok := r.tables[r.slowestTableID]; ok { + WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.spans.Len())) + if table, ok := r.spans.Get(r.slowestTableID); ok { slowestTableIDGauge. - WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.slowestTableID)) + WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.slowestTableID.TableID)) slowestTableStateGauge. WithLabelValues(cf.Namespace, cf.ID).Set(float64(table.State)) phyCkpTs := oracle.ExtractPhysical(table.Checkpoint.CheckpointTs) @@ -625,9 +658,9 @@ func (r *Manager) CollectMetrics() { metricAcceptScheduleTask.WithLabelValues("burstBalance").Add(float64(r.acceptBurstBalanceTask)) r.acceptBurstBalanceTask = 0 runningScheduleTaskGauge. - WithLabelValues(cf.Namespace, cf.ID).Set(float64(len(r.runningTasks))) + WithLabelValues(cf.Namespace, cf.ID).Set(float64(r.runningTasks.Len())) var stateCounters [6]int - for _, table := range r.tables { + r.spans.Ascend(func(span tablepb.Span, table *ReplicationSet) bool { switch table.State { case ReplicationSetStateUnknown: stateCounters[ReplicationSetStateUnknown]++ @@ -642,7 +675,8 @@ func (r *Manager) CollectMetrics() { case ReplicationSetStateRemoving: stateCounters[ReplicationSetStateRemoving]++ } - } + return true + }) for s, counter := range stateCounters { tableStateGauge. WithLabelValues(cf.Namespace, cf.ID, ReplicationSetState(s).String()). @@ -667,22 +701,6 @@ func (r *Manager) CleanMetrics() { metricAcceptScheduleTask.DeleteLabelValues("moveTable") metricAcceptScheduleTask.DeleteLabelValues("burstBalance") var stateCounters [6]int - for _, table := range r.tables { - switch table.State { - case ReplicationSetStateUnknown: - stateCounters[ReplicationSetStateUnknown]++ - case ReplicationSetStateAbsent: - stateCounters[ReplicationSetStateAbsent]++ - case ReplicationSetStatePrepare: - stateCounters[ReplicationSetStatePrepare]++ - case ReplicationSetStateCommit: - stateCounters[ReplicationSetStateCommit]++ - case ReplicationSetStateReplicating: - stateCounters[ReplicationSetStateReplicating]++ - case ReplicationSetStateRemoving: - stateCounters[ReplicationSetStateRemoving]++ - } - } for s := range stateCounters { tableStateGauge. DeleteLabelValues(cf.Namespace, cf.ID, ReplicationSetState(s).String()) @@ -698,10 +716,10 @@ func (r *Manager) CleanMetrics() { // SetReplicationSetForTests is only used in tests. func (r *Manager) SetReplicationSetForTests(rs *ReplicationSet) { - r.tables[rs.TableID] = rs + r.spans.ReplaceOrInsert(rs.Span, rs) } // GetReplicationSetForTests is only used in tests. -func (r *Manager) GetReplicationSetForTests() map[model.TableID]*ReplicationSet { - return r.tables +func (r *Manager) GetReplicationSetForTests() *spanz.Map[*ReplicationSet] { + return r.spans } diff --git a/cdc/scheduler/internal/v3/replication/replication_manager_test.go b/cdc/scheduler/internal/v3/replication/replication_manager_test.go index 621568d2425..41cec75e351 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -30,7 +31,9 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { addTableCh := make(chan int, 1) // Absent -> Prepare msgs, err := r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 1, CaptureID: "1", CheckpointTs: 1}, + AddTable: &AddTable{ + Span: spanz.TableIDToComparableSpan(1), CaptureID: "1", CheckpointTs: 1, + }, Accept: func() { addTableCh <- 1 close(addTableCh) @@ -44,7 +47,7 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 1, + Span: spanz.TableIDToComparableSpan(1), IsSecondary: true, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 1, @@ -54,12 +57,12 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { }, }, }, msgs[0]) - require.NotNil(t, r.runningTasks[1]) + require.NotNil(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(1))) require.Equal(t, 1, <-addTableCh) // Ignore if add the table again. msgs, err = r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 1, CaptureID: "1"}, + AddTable: &AddTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { t.Fatalf("must not accept") }, }}) require.Nil(t, err) @@ -73,8 +76,8 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { Response: &schedulepb.DispatchTableResponse_AddTable{ AddTable: &schedulepb.AddTableResponse{ Status: &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStatePrepared, + Span: spanz.TableIDToComparableSpan(1), + State: tablepb.TableStatePrepared, }, }, }, @@ -88,7 +91,7 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 1, + Span: spanz.TableIDToComparableSpan(1), IsSecondary: false, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 1, @@ -98,9 +101,10 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { }, }, }, msgs[0]) - require.Equal(t, ReplicationSetStateCommit, r.tables[1].State) - require.Equal(t, "1", r.tables[1].Primary) - require.False(t, r.tables[1].hasRole(RoleSecondary)) + require.Equal( + t, ReplicationSetStateCommit, r.spans.GetV(spanz.TableIDToComparableSpan(1)).State) + require.Equal(t, "1", r.spans.GetV(spanz.TableIDToComparableSpan(1)).Primary) + require.False(t, r.spans.GetV(spanz.TableIDToComparableSpan(1)).hasRole(RoleSecondary)) // Commit -> Replicating through heartbeat response. msgs, err = r.HandleMessage([]*schedulepb.Message{{ @@ -108,22 +112,23 @@ func TestReplicationManagerHandleAddTableTask(t *testing.T) { MsgType: schedulepb.MsgHeartbeatResponse, HeartbeatResponse: &schedulepb.HeartbeatResponse{ Tables: []tablepb.TableStatus{{ - TableID: 1, - State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(1), + State: tablepb.TableStateReplicating, }}, }, }}) require.Nil(t, err) require.Len(t, msgs, 0) - require.Equal(t, ReplicationSetStateReplicating, r.tables[1].State) - require.Equal(t, "1", r.tables[1].Primary) - require.False(t, r.tables[1].hasRole(RoleSecondary)) + require.Equal( + t, ReplicationSetStateReplicating, r.spans.GetV(spanz.TableIDToComparableSpan(1)).State) + require.Equal(t, "1", r.spans.GetV(spanz.TableIDToComparableSpan(1)).Primary) + require.False(t, r.spans.GetV(spanz.TableIDToComparableSpan(1)).hasRole(RoleSecondary)) // Handle task again to clear runningTasks msgs, err = r.HandleTasks(nil) require.Nil(t, err) require.Len(t, msgs, 0) - require.Nil(t, r.runningTasks[1]) + require.Nil(t, r.runningTasks.GetV(spanz.TableIDToComparableSpan(1))) } func TestReplicationManagerRemoveTable(t *testing.T) { @@ -134,23 +139,24 @@ func TestReplicationManagerRemoveTable(t *testing.T) { // Ignore remove table if there is no such table. msgs, err := r.HandleTasks([]*ScheduleTask{{ - RemoveTable: &RemoveTable{TableID: 1, CaptureID: "1"}, + RemoveTable: &RemoveTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { t.Fatal("must not accept") }, }}) require.Nil(t, err) require.Len(t, msgs, 0) // Add the table. - tbl, err := NewReplicationSet(1, 0, map[string]*tablepb.TableStatus{ - "1": {TableID: 1, State: tablepb.TableStateReplicating}, + span := spanz.TableIDToComparableSpan(1) + tbl, err := NewReplicationSet(span, 0, map[string]*tablepb.TableStatus{ + "1": {Span: span, State: tablepb.TableStateReplicating}, }, model.ChangeFeedID{}) require.Nil(t, err) require.Equal(t, ReplicationSetStateReplicating, tbl.State) - r.tables[1] = tbl + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(1), tbl) // Remove the table. msgs, err = r.HandleTasks([]*ScheduleTask{{ - RemoveTable: &RemoveTable{TableID: 1, CaptureID: "1"}, + RemoveTable: &RemoveTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { removeTableCh <- 1 close(removeTableCh) @@ -163,16 +169,16 @@ func TestReplicationManagerRemoveTable(t *testing.T) { MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: 1}, + RemoveTable: &schedulepb.RemoveTableRequest{Span: span}, }, }, }, msgs[0]) - require.NotNil(t, r.runningTasks[1]) + require.NotNil(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(1))) require.Equal(t, 1, <-removeTableCh) // Ignore if remove table again. msgs, err = r.HandleTasks([]*ScheduleTask{{ - RemoveTable: &RemoveTable{TableID: 1, CaptureID: "1"}, + RemoveTable: &RemoveTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { t.Fatalf("must not accept") }, }}) require.Nil(t, err) @@ -186,8 +192,8 @@ func TestReplicationManagerRemoveTable(t *testing.T) { Response: &schedulepb.DispatchTableResponse_RemoveTable{ RemoveTable: &schedulepb.RemoveTableResponse{ Status: &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStateStopping, + Span: span, + State: tablepb.TableStateStopping, }, }, }, @@ -202,20 +208,20 @@ func TestReplicationManagerRemoveTable(t *testing.T) { MsgType: schedulepb.MsgHeartbeatResponse, HeartbeatResponse: &schedulepb.HeartbeatResponse{ Tables: []tablepb.TableStatus{{ - TableID: 1, - State: tablepb.TableStateStopped, + Span: span, + State: tablepb.TableStateStopped, }}, }, }}) require.Nil(t, err) require.Len(t, msgs, 0) - require.Nil(t, r.tables[1]) + require.Nil(t, r.spans.GetV(spanz.TableIDToComparableSpan(1))) // Handle task again to clear runningTasks msgs, err = r.HandleTasks(nil) require.Nil(t, err) require.Len(t, msgs, 0) - require.Nil(t, r.runningTasks[1]) + require.Nil(t, r.runningTasks.GetV(spanz.TableIDToComparableSpan(1))) } func TestReplicationManagerMoveTable(t *testing.T) { @@ -229,23 +235,24 @@ func TestReplicationManagerMoveTable(t *testing.T) { // Ignore move table if it's not exist. msgs, err := r.HandleTasks([]*ScheduleTask{{ - MoveTable: &MoveTable{TableID: 1, DestCapture: dest}, + MoveTable: &MoveTable{Span: spanz.TableIDToComparableSpan(1), DestCapture: dest}, Accept: func() { t.Fatal("must not accept") }, }}) require.Nil(t, err) require.Len(t, msgs, 0) // Add the table. - tbl, err := NewReplicationSet(1, 0, map[string]*tablepb.TableStatus{ - source: {TableID: 1, State: tablepb.TableStateReplicating}, + span := spanz.TableIDToComparableSpan(1) + tbl, err := NewReplicationSet(span, 0, map[string]*tablepb.TableStatus{ + source: {Span: span, State: tablepb.TableStateReplicating}, }, model.ChangeFeedID{}) require.Nil(t, err) require.Equal(t, ReplicationSetStateReplicating, tbl.State) - r.tables[1] = tbl + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(1), tbl) // Replicating -> Prepare msgs, err = r.HandleTasks([]*ScheduleTask{{ - MoveTable: &MoveTable{TableID: 1, DestCapture: dest}, + MoveTable: &MoveTable{Span: spanz.TableIDToComparableSpan(1), DestCapture: dest}, Accept: func() { moveTableCh <- 1 close(moveTableCh) @@ -259,18 +266,18 @@ func TestReplicationManagerMoveTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 1, + Span: span, IsSecondary: true, }, }, }, }, msgs[0]) - require.NotNil(t, r.runningTasks[1]) + require.NotNil(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(1))) require.Equal(t, 1, <-moveTableCh) // Ignore if move table again. msgs, err = r.HandleTasks([]*ScheduleTask{{ - MoveTable: &MoveTable{TableID: 1, DestCapture: dest}, + MoveTable: &MoveTable{Span: spanz.TableIDToComparableSpan(1), DestCapture: dest}, Accept: func() { moveTableCh <- 1 close(moveTableCh) @@ -287,8 +294,8 @@ func TestReplicationManagerMoveTable(t *testing.T) { Response: &schedulepb.DispatchTableResponse_AddTable{ AddTable: &schedulepb.AddTableResponse{ Status: &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStatePrepared, + Span: span, + State: tablepb.TableStatePrepared, }, }, }, @@ -301,7 +308,7 @@ func TestReplicationManagerMoveTable(t *testing.T) { MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: 1}, + RemoveTable: &schedulepb.RemoveTableRequest{Span: span}, }, }, }, msgs[0]) @@ -313,8 +320,8 @@ func TestReplicationManagerMoveTable(t *testing.T) { MsgType: schedulepb.MsgHeartbeatResponse, HeartbeatResponse: &schedulepb.HeartbeatResponse{ Tables: []tablepb.TableStatus{{ - TableID: 1, - State: tablepb.TableStateStopped, + Span: span, + State: tablepb.TableStateStopped, }}, }, }}) @@ -326,7 +333,7 @@ func TestReplicationManagerMoveTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 1, + Span: span, IsSecondary: false, }, }, @@ -341,8 +348,8 @@ func TestReplicationManagerMoveTable(t *testing.T) { Response: &schedulepb.DispatchTableResponse_AddTable{ AddTable: &schedulepb.AddTableResponse{ Status: &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStateReplicating, + Span: span, + State: tablepb.TableStateReplicating, }, }, }, @@ -350,14 +357,15 @@ func TestReplicationManagerMoveTable(t *testing.T) { }}) require.Nil(t, err) require.Len(t, msgs, 0) - require.Equal(t, ReplicationSetStateReplicating, r.tables[1].State) - require.Equal(t, dest, r.tables[1].Primary) + require.Equal( + t, ReplicationSetStateReplicating, r.spans.GetV(spanz.TableIDToComparableSpan(1)).State) + require.Equal(t, dest, r.spans.GetV(spanz.TableIDToComparableSpan(1)).Primary) // Handle task again to clear runningTasks msgs, err = r.HandleTasks(nil) require.Nil(t, err) require.Len(t, msgs, 0) - require.Nil(t, r.runningTasks[1]) + require.Nil(t, r.runningTasks.GetV(spanz.TableIDToComparableSpan(1))) } func TestReplicationManagerBurstBalance(t *testing.T) { @@ -368,15 +376,17 @@ func TestReplicationManagerBurstBalance(t *testing.T) { // Burst balance is not limited by maxTaskConcurrency. msgs, err := r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 1, CaptureID: "0", CheckpointTs: 1}, + AddTable: &AddTable{ + Span: spanz.TableIDToComparableSpan(1), CaptureID: "0", CheckpointTs: 1, + }, }, { BurstBalance: &BurstBalance{ AddTables: []AddTable{{ - TableID: 1, CaptureID: "1", CheckpointTs: 1, + Span: spanz.TableIDToComparableSpan(1), CaptureID: "1", CheckpointTs: 1, }, { - TableID: 2, CaptureID: "2", CheckpointTs: 1, + Span: spanz.TableIDToComparableSpan(2), CaptureID: "2", CheckpointTs: 1, }, { - TableID: 3, CaptureID: "3", CheckpointTs: 1, + Span: spanz.TableIDToComparableSpan(3), CaptureID: "3", CheckpointTs: 1, }}, }, Accept: func() { @@ -395,7 +405,7 @@ func TestReplicationManagerBurstBalance(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: tableID, + Span: spanz.TableIDToComparableSpan(tableID), IsSecondary: true, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 1, @@ -405,28 +415,30 @@ func TestReplicationManagerBurstBalance(t *testing.T) { }, }, }, msgs) - require.Contains(t, r.tables, tableID) - require.Contains(t, r.runningTasks, tableID) + require.True(t, r.spans.Has(spanz.TableIDToComparableSpan(tableID))) + require.True(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(tableID))) } // Add a new table. - r.tables[5], err = NewReplicationSet(5, 0, map[string]*tablepb.TableStatus{ - "5": {TableID: 5, State: tablepb.TableStateReplicating}, + span := spanz.TableIDToComparableSpan(5) + table5, err := NewReplicationSet(span, 0, map[string]*tablepb.TableStatus{ + "5": {Span: span, State: tablepb.TableStateReplicating}, }, model.ChangeFeedID{}) require.Nil(t, err) + r.spans.ReplaceOrInsert(span, table5) // More burst balance is still allowed. msgs, err = r.HandleTasks([]*ScheduleTask{{ BurstBalance: &BurstBalance{ AddTables: []AddTable{{ - TableID: 4, CaptureID: "4", CheckpointTs: 2, + Span: spanz.TableIDToComparableSpan(4), CaptureID: "4", CheckpointTs: 2, }, { - TableID: 1, CaptureID: "0", CheckpointTs: 2, + Span: spanz.TableIDToComparableSpan(1), CaptureID: "0", CheckpointTs: 2, }}, RemoveTables: []RemoveTable{{ - TableID: 5, CaptureID: "5", + Span: spanz.TableIDToComparableSpan(5), CaptureID: "5", }, { - TableID: 1, CaptureID: "0", + Span: spanz.TableIDToComparableSpan(1), CaptureID: "0", }}, }, Accept: func() { @@ -442,7 +454,7 @@ func TestReplicationManagerBurstBalance(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 4, + Span: spanz.TableIDToComparableSpan(4), IsSecondary: true, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 2, @@ -458,7 +470,7 @@ func TestReplicationManagerBurstBalance(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ RemoveTable: &schedulepb.RemoveTableRequest{ - TableID: 5, + Span: spanz.TableIDToComparableSpan(5), }, }, }, @@ -473,22 +485,26 @@ func TestReplicationManagerBurstBalanceMoveTables(t *testing.T) { var err error // Two tables in "1". - r.tables[1], err = NewReplicationSet(1, 0, map[string]*tablepb.TableStatus{ - "1": {TableID: 1, State: tablepb.TableStateReplicating}, + span := spanz.TableIDToComparableSpan(1) + table, err := NewReplicationSet(span, 0, map[string]*tablepb.TableStatus{ + "1": {Span: span, State: tablepb.TableStateReplicating}, }, model.ChangeFeedID{}) require.Nil(t, err) - r.tables[2], err = NewReplicationSet(2, 0, map[string]*tablepb.TableStatus{ + r.spans.ReplaceOrInsert(span, table) + span2 := spanz.TableIDToComparableSpan(2) + table2, err := NewReplicationSet(span2, 0, map[string]*tablepb.TableStatus{ "1": { - TableID: 2, State: tablepb.TableStateReplicating, + Span: span2, State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, }, }, model.ChangeFeedID{}) require.Nil(t, err) + r.spans.ReplaceOrInsert(span2, table2) msgs, err := r.HandleTasks([]*ScheduleTask{{ BurstBalance: &BurstBalance{ MoveTables: []MoveTable{{ - TableID: 2, DestCapture: "2", + Span: spanz.TableIDToComparableSpan(2), DestCapture: "2", }}, }, Accept: func() { @@ -504,15 +520,15 @@ func TestReplicationManagerBurstBalanceMoveTables(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 2, + Span: span2, IsSecondary: true, Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, }, }, }, }, msgs) - require.Contains(t, r.tables, model.TableID(2)) - require.Contains(t, r.runningTasks, model.TableID(2)) + require.True(t, r.spans.Has(span2)) + require.True(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(2))) } func TestReplicationManagerMaxTaskConcurrency(t *testing.T) { @@ -522,7 +538,7 @@ func TestReplicationManagerMaxTaskConcurrency(t *testing.T) { addTableCh := make(chan int, 1) msgs, err := r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 1, CaptureID: "1"}, + AddTable: &AddTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { addTableCh <- 1 close(addTableCh) @@ -536,18 +552,18 @@ func TestReplicationManagerMaxTaskConcurrency(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: 1, + Span: spanz.TableIDToComparableSpan(1), IsSecondary: true, }, }, }, }, msgs[0]) - require.NotNil(t, r.runningTasks[1]) + require.NotNil(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(1))) require.Equal(t, 1, <-addTableCh) // No more tasks allowed. msgs, err = r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 2, CaptureID: "1"}, + AddTable: &AddTable{Span: spanz.TableIDToComparableSpan(2), CaptureID: "1"}, Accept: func() { t.Fatal("must not accept") }, @@ -560,11 +576,12 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { t.Parallel() r := NewReplicationManager(1, model.ChangeFeedID{}) - rs, err := NewReplicationSet(model.TableID(1), model.Ts(10), + span := spanz.TableIDToComparableSpan(1) + rs, err := NewReplicationSet(span, model.Ts(10), map[model.CaptureID]*tablepb.TableStatus{ "1": { - TableID: model.TableID(1), - State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(1), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(10), ResolvedTs: model.Ts(20), @@ -572,13 +589,14 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, }, model.ChangeFeedID{}) require.NoError(t, err) - r.tables[model.TableID(1)] = rs + r.spans.ReplaceOrInsert(span, rs) - rs, err = NewReplicationSet(model.TableID(2), model.Ts(15), + span2 := spanz.TableIDToComparableSpan(2) + rs, err = NewReplicationSet(span2, model.Ts(15), map[model.CaptureID]*tablepb.TableStatus{ "2": { - TableID: model.TableID(2), - State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(2), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(15), ResolvedTs: model.Ts(30), @@ -586,9 +604,9 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, }, model.ChangeFeedID{}) require.NoError(t, err) - r.tables[model.TableID(2)] = rs + r.spans.ReplaceOrInsert(span2, rs) - // all table is replicating + // all tables are replicating currentTables := []model.TableID{1, 2} checkpoint, resolved := r.AdvanceCheckpoint(currentTables, time.Now()) require.Equal(t, model.Ts(10), checkpoint) @@ -600,19 +618,20 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { require.Equal(t, checkpointCannotProceed, checkpoint) require.Equal(t, checkpointCannotProceed, resolved) - rs, err = NewReplicationSet(model.TableID(3), model.Ts(5), + span3 := spanz.TableIDToComparableSpan(3) + rs, err = NewReplicationSet(span3, model.Ts(5), map[model.CaptureID]*tablepb.TableStatus{ "1": { - TableID: model.TableID(3), - State: tablepb.TableStateReplicating, + Span: spanz.TableIDToComparableSpan(3), + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(5), ResolvedTs: model.Ts(40), }, }, "2": { - TableID: model.TableID(3), - State: tablepb.TableStatePreparing, + Span: spanz.TableIDToComparableSpan(3), + State: tablepb.TableStatePreparing, Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(5), ResolvedTs: model.Ts(40), @@ -620,17 +639,18 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, }, model.ChangeFeedID{}) require.NoError(t, err) - r.tables[model.TableID(3)] = rs + r.spans.ReplaceOrInsert(span3, rs) checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now()) require.Equal(t, model.Ts(5), checkpoint) require.Equal(t, model.Ts(20), resolved) currentTables = append(currentTables, 4) - rs, err = NewReplicationSet(model.TableID(4), model.Ts(3), + span4 := spanz.TableIDToComparableSpan(4) + rs, err = NewReplicationSet(span4, model.Ts(3), map[model.CaptureID]*tablepb.TableStatus{ "1": { - TableID: model.TableID(4), - State: tablepb.TableStatePrepared, + Span: spanz.TableIDToComparableSpan(4), + State: tablepb.TableStatePrepared, Checkpoint: tablepb.Checkpoint{ CheckpointTs: model.Ts(3), ResolvedTs: model.Ts(10), @@ -638,10 +658,48 @@ func TestReplicationManagerAdvanceCheckpoint(t *testing.T) { }, }, model.ChangeFeedID{}) require.NoError(t, err) - r.tables[model.TableID(4)] = rs + r.spans.ReplaceOrInsert(span4, rs) + checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now()) + require.Equal(t, model.Ts(3), checkpoint) + require.Equal(t, model.Ts(10), resolved) + + // Split table 5 into 2 spans. + currentTables = append(currentTables, 5) + span5_1 := spanz.TableIDToComparableSpan(5) + span5_1.EndKey = append(span5_1.StartKey, 0) + span5_2 := spanz.TableIDToComparableSpan(5) + span5_2.StartKey = append(span5_2.StartKey, 0) + for _, span := range []tablepb.Span{span5_1, span5_2} { + rs, err = NewReplicationSet(span, model.Ts(3), + map[model.CaptureID]*tablepb.TableStatus{ + "1": { + Span: span, + State: tablepb.TableStatePrepared, + Checkpoint: tablepb.Checkpoint{ + CheckpointTs: model.Ts(3), + ResolvedTs: model.Ts(10), + }, + }, + }, model.ChangeFeedID{}) + require.NoError(t, err) + r.spans.ReplaceOrInsert(span, rs) + } checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now()) require.Equal(t, model.Ts(3), checkpoint) require.Equal(t, model.Ts(10), resolved) + + // The start span is missing + rs5_1, _ := r.spans.Delete(span5_1) + checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now()) + require.Equal(t, checkpointCannotProceed, checkpoint) + require.Equal(t, checkpointCannotProceed, resolved) + + // The end span is missing + r.spans.ReplaceOrInsert(span5_1, rs5_1) + r.spans.Delete(span5_2) + checkpoint, resolved = r.AdvanceCheckpoint(currentTables, time.Now()) + require.Equal(t, checkpointCannotProceed, checkpoint) + require.Equal(t, checkpointCannotProceed, resolved) } func TestReplicationManagerHandleCaptureChanges(t *testing.T) { @@ -649,37 +707,47 @@ func TestReplicationManagerHandleCaptureChanges(t *testing.T) { r := NewReplicationManager(1, model.ChangeFeedID{}) init := map[model.CaptureID][]tablepb.TableStatus{ - "1": {{TableID: 1, State: tablepb.TableStateReplicating}}, - "2": {{TableID: 2, State: tablepb.TableStateReplicating}}, + "1": {{Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating}}, + "2": {{Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStateReplicating}}, "3": { - {TableID: 3, State: tablepb.TableStateReplicating}, - {TableID: 2, State: tablepb.TableStatePreparing}, + {Span: spanz.TableIDToComparableSpan(3), State: tablepb.TableStateReplicating}, + {Span: spanz.TableIDToComparableSpan(2), State: tablepb.TableStatePreparing}, }, - "4": {{TableID: 4, State: tablepb.TableStateStopping}}, - "5": {{TableID: 5, State: tablepb.TableStateStopped}}, + "4": {{Span: spanz.TableIDToComparableSpan(4), State: tablepb.TableStateStopping}}, + "5": {{Span: spanz.TableIDToComparableSpan(5), State: tablepb.TableStateStopped}}, } msgs, err := r.HandleCaptureChanges(init, nil, 0) require.Nil(t, err) require.Len(t, msgs, 0) - require.Len(t, r.tables, 5) - require.Equal(t, ReplicationSetStateReplicating, r.tables[1].State) - require.Equal(t, ReplicationSetStatePrepare, r.tables[2].State) - require.Equal(t, ReplicationSetStateReplicating, r.tables[3].State) - require.Equal(t, ReplicationSetStateRemoving, r.tables[4].State) - require.Equal(t, ReplicationSetStateAbsent, r.tables[5].State) + require.Equal(t, r.spans.Len(), 5) + require.Equal( + t, ReplicationSetStateReplicating, r.spans.GetV(spanz.TableIDToComparableSpan(1)).State) + require.Equal( + t, ReplicationSetStatePrepare, r.spans.GetV(spanz.TableIDToComparableSpan(2)).State) + require.Equal( + t, ReplicationSetStateReplicating, r.spans.GetV(spanz.TableIDToComparableSpan(3)).State) + require.Equal( + t, ReplicationSetStateRemoving, r.spans.GetV(spanz.TableIDToComparableSpan(4)).State) + require.Equal( + t, ReplicationSetStateAbsent, r.spans.GetV(spanz.TableIDToComparableSpan(5)).State) removed := map[string][]tablepb.TableStatus{ - "1": {{TableID: 1, State: tablepb.TableStateReplicating}}, + "1": {{Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStateReplicating}}, } msgs, err = r.HandleCaptureChanges(nil, removed, 0) require.Nil(t, err) require.Len(t, msgs, 0) - require.Len(t, r.tables, 5) - require.Equal(t, ReplicationSetStateAbsent, r.tables[1].State) - require.Equal(t, ReplicationSetStatePrepare, r.tables[2].State) - require.Equal(t, ReplicationSetStateReplicating, r.tables[3].State) - require.Equal(t, ReplicationSetStateRemoving, r.tables[4].State) - require.Equal(t, ReplicationSetStateAbsent, r.tables[5].State) + require.Equal(t, r.spans.Len(), 5) + require.Equal( + t, ReplicationSetStateAbsent, r.spans.GetV(spanz.TableIDToComparableSpan(1)).State) + require.Equal( + t, ReplicationSetStatePrepare, r.spans.GetV(spanz.TableIDToComparableSpan(2)).State) + require.Equal( + t, ReplicationSetStateReplicating, r.spans.GetV(spanz.TableIDToComparableSpan(3)).State) + require.Equal( + t, ReplicationSetStateRemoving, r.spans.GetV(spanz.TableIDToComparableSpan(4)).State) + require.Equal( + t, ReplicationSetStateAbsent, r.spans.GetV(spanz.TableIDToComparableSpan(5)).State) } func TestReplicationManagerHandleCaptureChangesDuringAddTable(t *testing.T) { @@ -689,98 +757,97 @@ func TestReplicationManagerHandleCaptureChangesDuringAddTable(t *testing.T) { addTableCh := make(chan int, 1) msgs, err := r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 1, CaptureID: "1"}, + AddTable: &AddTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { addTableCh <- 1 }, }}) require.Nil(t, err) require.Len(t, msgs, 1) - require.NotNil(t, r.runningTasks[1]) + require.NotNil(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(1))) require.Equal(t, 1, <-addTableCh) removed := map[string][]tablepb.TableStatus{ - "1": {{TableID: 1, State: tablepb.TableStatePreparing}}, + "1": {{Span: spanz.TableIDToComparableSpan(1), State: tablepb.TableStatePreparing}}, } msgs, err = r.HandleCaptureChanges(nil, removed, 0) require.Nil(t, err) require.Len(t, msgs, 0) - require.Len(t, r.tables, 1) - require.Equal(t, ReplicationSetStateAbsent, r.tables[1].State) - require.Nil(t, r.runningTasks[1]) + require.Equal(t, r.spans.Len(), 1) + require.Equal( + t, ReplicationSetStateAbsent, r.spans.GetV(spanz.TableIDToComparableSpan(1)).State) + require.Nil(t, r.runningTasks.GetV(spanz.TableIDToComparableSpan(1))) // New task must be accepted. msgs, err = r.HandleTasks([]*ScheduleTask{{ - AddTable: &AddTable{TableID: 1, CaptureID: "1"}, + AddTable: &AddTable{Span: spanz.TableIDToComparableSpan(1), CaptureID: "1"}, Accept: func() { addTableCh <- 1 }, }}) require.Nil(t, err) require.Len(t, msgs, 1) - require.NotNil(t, r.runningTasks[1]) + require.NotNil(t, r.runningTasks.Has(spanz.TableIDToComparableSpan(1))) require.Equal(t, 1, <-addTableCh) } func TestLogSlowTableInfo(t *testing.T) { t.Parallel() r := NewReplicationManager(1, model.ChangeFeedID{}) - r.tables[1] = &ReplicationSet{ - TableID: 1, + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(1), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(1), Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, State: ReplicationSetStateReplicating, - } - r.tables[2] = &ReplicationSet{ - TableID: 2, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(2), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(2), Checkpoint: tablepb.Checkpoint{CheckpointTs: 2}, State: ReplicationSetStatePrepare, - } - r.tables[3] = &ReplicationSet{ - TableID: 3, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(3), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(3), Checkpoint: tablepb.Checkpoint{CheckpointTs: 3}, State: ReplicationSetStatePrepare, - } - currentTables := []model.TableID{1, 2, 3} - r.logSlowTableInfo(currentTables, time.Now()) + }) + r.logSlowTableInfo(time.Now()) // make sure all tables are will be pop out from heal after logged require.Equal(t, r.slowTableHeap.Len(), 0) - r.tables[4] = &ReplicationSet{ - TableID: 4, + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(4), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(4), Checkpoint: tablepb.Checkpoint{CheckpointTs: 4}, State: ReplicationSetStatePrepare, - } - r.tables[5] = &ReplicationSet{ - TableID: 5, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(5), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(5), Checkpoint: tablepb.Checkpoint{CheckpointTs: 5}, State: ReplicationSetStatePrepare, - } - r.tables[6] = &ReplicationSet{ - TableID: 6, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(6), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(6), Checkpoint: tablepb.Checkpoint{CheckpointTs: 6}, State: ReplicationSetStatePrepare, - } - r.tables[7] = &ReplicationSet{ - TableID: 7, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(7), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(7), Checkpoint: tablepb.Checkpoint{CheckpointTs: 7}, State: ReplicationSetStatePrepare, - } - r.tables[8] = &ReplicationSet{ - TableID: 8, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(8), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(8), Checkpoint: tablepb.Checkpoint{CheckpointTs: 8}, State: ReplicationSetStatePrepare, - } - r.tables[9] = &ReplicationSet{ - TableID: 9, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(9), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(9), Checkpoint: tablepb.Checkpoint{CheckpointTs: 9}, State: ReplicationSetStatePrepare, - } - r.tables[10] = &ReplicationSet{ - TableID: 10, + }) + r.spans.ReplaceOrInsert(spanz.TableIDToComparableSpan(1), &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(10), Checkpoint: tablepb.Checkpoint{CheckpointTs: 10}, State: ReplicationSetStatePrepare, - } - currentTables = []model.TableID{1, 2, 3, 4, 5, 6, 7, 8, 9, 10} - r.logSlowTableInfo(currentTables, time.Now()) + }) + r.logSlowTableInfo(time.Now()) // make sure the slowTableHeap's capacity will not extend require.Equal(t, cap(r.slowTableHeap), 8) } diff --git a/cdc/scheduler/internal/v3/replication/replication_set.go b/cdc/scheduler/internal/v3/replication/replication_set.go index 327d89df389..24078dbf1d8 100644 --- a/cdc/scheduler/internal/v3/replication/replication_set.go +++ b/cdc/scheduler/internal/v3/replication/replication_set.go @@ -127,7 +127,7 @@ func (r Role) MarshalJSON() ([]byte, error) { // ReplicationSet is a state machine that manages replication states. type ReplicationSet struct { //nolint:revive Changefeed model.ChangeFeedID - TableID model.TableID + Span tablepb.Span State ReplicationSetState // Primary is the capture ID that is currently replicating the table. Primary model.CaptureID @@ -141,14 +141,14 @@ type ReplicationSet struct { //nolint:revive // NewReplicationSet returns a new replication set. func NewReplicationSet( - tableID model.TableID, + span tablepb.Span, checkpoint model.Ts, tableStatus map[model.CaptureID]*tablepb.TableStatus, changefeed model.ChangeFeedID, ) (*ReplicationSet, error) { r := &ReplicationSet{ Changefeed: changefeed, - TableID: tableID, + Span: span, Captures: make(map[string]Role), Checkpoint: tablepb.Checkpoint{ CheckpointTs: checkpoint, @@ -159,7 +159,7 @@ func NewReplicationSet( stoppingCount := 0 committed := false for captureID, table := range tableStatus { - if r.TableID != table.TableID { + if !r.Span.Eq(&table.Span) { return nil, r.inconsistentError(table, captureID, "schedulerv3: table id inconsistent") } @@ -202,7 +202,7 @@ func NewReplicationSet( // proceeding further scheduling. log.Warn("schedulerv3: found a stopping capture during initializing", zap.Any("replicationSet", r), - zap.Int64("tableID", table.TableID), + zap.Int64("tableID", table.Span.TableID), zap.Any("status", tableStatus)) err := r.setCapture(captureID, RoleUndetermined) if err != nil { @@ -215,7 +215,7 @@ func NewReplicationSet( default: log.Warn("schedulerv3: unknown table state", zap.Any("replicationSet", r), - zap.Int64("tableID", table.TableID), + zap.Int64("tableID", table.Span.TableID), zap.Any("status", tableStatus)) } } @@ -324,7 +324,7 @@ func (r *ReplicationSet) inconsistentError( }...) log.L().WithOptions(zap.AddCallerSkip(1)).Error(msg, fields...) return cerror.ErrReplicationSetInconsistent.GenWithStackByArgs( - fmt.Sprintf("tableID %d, %s", r.TableID, msg)) + fmt.Sprintf("tableID %d, %s", r.Span.TableID, msg)) } func (r *ReplicationSet) multiplePrimaryError( @@ -337,14 +337,14 @@ func (r *ReplicationSet) multiplePrimaryError( }...) log.L().WithOptions(zap.AddCallerSkip(1)).Error(msg, fields...) return cerror.ErrReplicationSetMultiplePrimaryError.GenWithStackByArgs( - fmt.Sprintf("tableID %d, %s", r.TableID, msg)) + fmt.Sprintf("tableID %d, %s", r.Span.TableID, msg)) } // checkInvariant ensures ReplicationSet invariant is hold. func (r *ReplicationSet) checkInvariant( input *tablepb.TableStatus, captureID model.CaptureID, ) error { - if r.TableID != input.TableID { + if !r.Span.Eq(&input.Span) { return r.inconsistentError(input, captureID, "schedulerv3: tableID must be the same") } @@ -463,7 +463,7 @@ func (r *ReplicationSet) pollOnPrepare( DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: r.Span, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -540,7 +540,9 @@ func (r *ReplicationSet) pollOnCommit( MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, + RemoveTable: &schedulepb.RemoveTableRequest{ + Span: r.Span, + }, }, }, }, false, nil @@ -575,7 +577,7 @@ func (r *ReplicationSet) pollOnCommit( DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: r.Span, IsSecondary: false, Checkpoint: r.Checkpoint, }, @@ -615,7 +617,7 @@ func (r *ReplicationSet) pollOnCommit( DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: r.Span, IsSecondary: false, Checkpoint: r.Checkpoint, }, @@ -658,7 +660,9 @@ func (r *ReplicationSet) pollOnCommit( MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, + RemoveTable: &schedulepb.RemoveTableRequest{ + Span: r.Span, + }, }, }, }, false, nil @@ -754,7 +758,9 @@ func (r *ReplicationSet) pollOnRemoving( MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, + RemoveTable: &schedulepb.RemoveTableRequest{ + Span: r.Span, + }, }, }, }, false, nil @@ -797,7 +803,7 @@ func (r *ReplicationSet) handleAddTable( // Ignore add table if it's not in Absent state. if r.State != ReplicationSetStateAbsent { log.Warn("schedulerv3: add table is ignored", - zap.Any("replicationSet", r), zap.Int64("tableID", r.TableID)) + zap.Any("replicationSet", r), zap.Int64("tableID", r.Span.TableID)) return nil, nil } oldState := r.State @@ -810,7 +816,7 @@ func (r *ReplicationSet) handleAddTable( zap.Any("replicationSet", r), zap.Stringer("old", oldState), zap.Stringer("new", r.State)) status := tablepb.TableStatus{ - TableID: r.TableID, + Span: r.Span, State: tablepb.TableStateAbsent, Checkpoint: tablepb.Checkpoint{}, } @@ -823,7 +829,7 @@ func (r *ReplicationSet) handleMoveTable( // Ignore move table if it has been removed already. if r.hasRemoved() { log.Warn("schedulerv3: move table is ignored", - zap.Any("replicationSet", r), zap.Int64("tableID", r.TableID)) + zap.Any("replicationSet", r), zap.Int64("tableID", r.Span.TableID)) return nil, nil } // Ignore move table if @@ -831,7 +837,7 @@ func (r *ReplicationSet) handleMoveTable( // 2) the dest capture is the primary. if r.State != ReplicationSetStateReplicating || r.Primary == dest { log.Warn("schedulerv3: move table is ignored", - zap.Any("replicationSet", r), zap.Int64("tableID", r.TableID)) + zap.Any("replicationSet", r), zap.Int64("tableID", r.Span.TableID)) return nil, nil } oldState := r.State @@ -844,7 +850,7 @@ func (r *ReplicationSet) handleMoveTable( zap.Any("replicationSet", r), zap.Stringer("old", oldState), zap.Stringer("new", r.State)) status := tablepb.TableStatus{ - TableID: r.TableID, + Span: r.Span, State: tablepb.TableStateAbsent, Checkpoint: tablepb.Checkpoint{}, } @@ -855,13 +861,13 @@ func (r *ReplicationSet) handleRemoveTable() ([]*schedulepb.Message, error) { // Ignore remove table if it has been removed already. if r.hasRemoved() { log.Warn("schedulerv3: remove table is ignored", - zap.Any("replicationSet", r), zap.Int64("tableID", r.TableID)) + zap.Any("replicationSet", r), zap.Int64("tableID", r.Span.TableID)) return nil, nil } // Ignore remove table if it's not in Replicating state. if r.State != ReplicationSetStateReplicating { log.Warn("schedulerv3: remove table is ignored", - zap.Any("replicationSet", r), zap.Int64("tableID", r.TableID)) + zap.Any("replicationSet", r), zap.Int64("tableID", r.Span.TableID)) return nil, nil } oldState := r.State @@ -870,8 +876,8 @@ func (r *ReplicationSet) handleRemoveTable() ([]*schedulepb.Message, error) { zap.Any("replicationSet", r), zap.Stringer("old", oldState), zap.Stringer("new", r.State)) status := tablepb.TableStatus{ - TableID: r.TableID, - State: tablepb.TableStateReplicating, + Span: r.Span, + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: r.Checkpoint.CheckpointTs, ResolvedTs: r.Checkpoint.ResolvedTs, @@ -899,8 +905,8 @@ func (r *ReplicationSet) handleCaptureShutdown( } // The capture has shutdown, the table has stopped. status := tablepb.TableStatus{ - TableID: r.TableID, - State: tablepb.TableStateStopped, + Span: r.Span, + State: tablepb.TableStateStopped, } msgs, err := r.poll(&status, captureID) return msgs, true, errors.Trace(err) diff --git a/cdc/scheduler/internal/v3/replication/replication_set_test.go b/cdc/scheduler/internal/v3/replication/replication_set_test.go index 7ac098ba7b4..9cff235a9a8 100644 --- a/cdc/scheduler/internal/v3/replication/replication_set_test.go +++ b/cdc/scheduler/internal/v3/replication/replication_set_test.go @@ -16,7 +16,6 @@ package replication import ( "container/heap" "encoding/json" - "fmt" "math/rand" "testing" "time" @@ -24,6 +23,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/schedulepb" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -213,7 +213,8 @@ func TestNewReplicationSet(t *testing.T) { status := tc.tableStatus checkpoint := tc.checkpoint - output, err := NewReplicationSet(0, checkpoint, status, model.ChangeFeedID{}) + span := tablepb.Span{TableID: 0} + output, err := NewReplicationSet(span, checkpoint, status, model.ChangeFeedID{}) if set == nil { require.Errorf(t, err, "%d", id) } else { @@ -252,17 +253,18 @@ func TestReplicationSetPoll(t *testing.T) { status := make(map[string]*tablepb.TableStatus) for id, state := range states { status[id] = &tablepb.TableStatus{ - TableID: 1, + Span: tablepb.Span{TableID: 1}, State: state, Checkpoint: tablepb.Checkpoint{}, } } - r, _ := NewReplicationSet(1, 0, status, model.ChangeFeedID{}) + span := tablepb.Span{TableID: 1} + r, _ := NewReplicationSet(span, 0, status, model.ChangeFeedID{}) var tableStates []int for state := range tablepb.TableState_name { tableStates = append(tableStates, int(state)) } - input := &tablepb.TableStatus{TableID: model.TableID(1)} + input := &tablepb.TableStatus{Span: tablepb.Span{TableID: model.TableID(1)}} iterPermutation(tableStates, func(tableStateSequence []int) { t.Logf("test %d, %v, %v", seed, status, tableStateSequence) for _, state := range tableStateSequence { @@ -288,9 +290,10 @@ func TestReplicationSetPollUnknownCapture(t *testing.T) { t.Parallel() tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, map[model.CaptureID]*tablepb.TableStatus{ + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, map[model.CaptureID]*tablepb.TableStatus{ "1": { - TableID: tableID, + Span: tablepb.Span{TableID: tableID}, State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{}, }, @@ -298,22 +301,22 @@ func TestReplicationSetPollUnknownCapture(t *testing.T) { require.Nil(t, err) msgs, err := r.poll(&tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }, "unknown") require.Nil(t, msgs) require.Nil(t, err) msgs, err = r.poll(&tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateAbsent, }, "unknown") require.Len(t, msgs, 0) require.Nil(t, err) msgs, err = r.poll(&tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }, "unknown") require.Len(t, msgs, 0) require.Nil(t, err) @@ -324,7 +327,8 @@ func TestReplicationSetAddTable(t *testing.T) { from := "1" tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, nil, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, nil, model.ChangeFeedID{}) require.Nil(t, err) // Absent -> Prepare @@ -337,7 +341,7 @@ func TestReplicationSetAddTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -354,8 +358,8 @@ func TestReplicationSetAddTable(t *testing.T) { // AddTableRequest is lost somehow, send AddTableRequest again. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -365,7 +369,7 @@ func TestReplicationSetAddTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -377,8 +381,8 @@ func TestReplicationSetAddTable(t *testing.T) { // Prepare is in-progress. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePreparing, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePreparing, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -387,8 +391,8 @@ func TestReplicationSetAddTable(t *testing.T) { // Prepare -> Commit. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -398,7 +402,7 @@ func TestReplicationSetAddTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: false, Checkpoint: r.Checkpoint, }, @@ -410,8 +414,8 @@ func TestReplicationSetAddTable(t *testing.T) { require.False(t, r.hasRole(RoleSecondary)) // The secondary AddTable request may be lost. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -421,7 +425,7 @@ func TestReplicationSetAddTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: false, Checkpoint: r.Checkpoint, }, @@ -434,8 +438,8 @@ func TestReplicationSetAddTable(t *testing.T) { // Commit -> Replicating msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -445,8 +449,8 @@ func TestReplicationSetAddTable(t *testing.T) { // Replicating -> Replicating msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 3, ResolvedTs: 4, @@ -468,7 +472,8 @@ func TestReplicationSetRemoveTable(t *testing.T) { from := "1" tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, nil, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, nil, model.ChangeFeedID{}) require.Nil(t, err) // Ignore removing table if it's not in replicating. @@ -489,7 +494,9 @@ func TestReplicationSetRemoveTable(t *testing.T) { MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, + RemoveTable: &schedulepb.RemoveTableRequest{ + Span: tablepb.Span{TableID: r.Span.TableID}, + }, }, }, }, msgs[0]) @@ -503,8 +510,8 @@ func TestReplicationSetRemoveTable(t *testing.T) { // Removing is in-progress. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopping, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopping, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -514,8 +521,8 @@ func TestReplicationSetRemoveTable(t *testing.T) { // Removed if the table is absent. rClone := clone(r) msgs, err = rClone.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -524,8 +531,8 @@ func TestReplicationSetRemoveTable(t *testing.T) { // Removed if the table is stopped. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -546,7 +553,8 @@ func TestReplicationSetMoveTable(t *testing.T) { t.Parallel() tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, nil, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, nil, model.ChangeFeedID{}) require.Nil(t, err) source := "1" @@ -572,7 +580,7 @@ func TestReplicationSetMoveTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -590,8 +598,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Source primary sends heartbeat response msgs, err = r.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 1, ResolvedTs: 1, @@ -606,8 +614,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // AddTableRequest is lost somehow, send AddTableRequest again. msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -617,7 +625,7 @@ func TestReplicationSetMoveTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -629,8 +637,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Prepare -> Commit. msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -639,7 +647,7 @@ func TestReplicationSetMoveTable(t *testing.T) { MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, + RemoveTable: &schedulepb.RemoveTableRequest{Span: r.Span}, }, }, }, msgs[0]) @@ -649,8 +657,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Source updates it's table status msgs, err = r.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 2, ResolvedTs: 3, @@ -663,7 +671,7 @@ func TestReplicationSetMoveTable(t *testing.T) { MsgType: schedulepb.MsgDispatchTableRequest, DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_RemoveTable{ - RemoveTable: &schedulepb.RemoveTableRequest{TableID: r.TableID}, + RemoveTable: &schedulepb.RemoveTableRequest{Span: r.Span}, }, }, }, msgs[0]) @@ -677,8 +685,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Removing source is in-progress. msgs, err = r.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopping, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopping, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 3, ResolvedTs: 3, @@ -697,8 +705,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Source is removed. rClone := clone(r) msgs, err = r.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopped, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 3, ResolvedTs: 4, @@ -712,7 +720,7 @@ func TestReplicationSetMoveTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: false, Checkpoint: r.Checkpoint, }, @@ -730,8 +738,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Source stopped message is lost somehow. // rClone has checkpoint ts 3, resolved ts 3 msgs, err = rClone.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -741,7 +749,7 @@ func TestReplicationSetMoveTable(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: false, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 3, @@ -761,8 +769,8 @@ func TestReplicationSetMoveTable(t *testing.T) { // Commit -> Replicating msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -777,7 +785,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { from := "1" tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, nil, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, nil, model.ChangeFeedID{}) require.Nil(t, err) // Add table, Absent -> Prepare @@ -790,7 +799,7 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -816,8 +825,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { // Add table, Prepare -> Commit msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -840,8 +849,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { // Add table, Commit -> Replicating msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -910,8 +919,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { // Move table, Prepare -> Commit msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -932,7 +941,7 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: false, Checkpoint: r.Checkpoint, }, @@ -972,8 +981,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { t.Run("OriginalPrimaryReplicating", func(t *testing.T) { rClone1 := clone(rClone) msgs, err = rClone1.handleTableStatus(rClone1.Primary, &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: 1}, + State: tablepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -988,8 +997,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { t.Run("OriginalPrimaryStopped", func(t *testing.T) { rClone1 := clone(rClone) msgs, err = rClone1.handleTableStatus(rClone1.Primary, &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: 1}, + State: tablepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1004,8 +1013,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { t.Run("OriginalPrimaryAbsent", func(t *testing.T) { rClone1 := clone(rClone) msgs, err = rClone1.handleTableStatus(rClone1.Primary, &tablepb.TableStatus{ - TableID: 1, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: 1}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -1015,7 +1024,7 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { DispatchTableRequest: &schedulepb.DispatchTableRequest{ Request: &schedulepb.DispatchTableRequest_AddTable{ AddTable: &schedulepb.AddTableRequest{ - TableID: r.TableID, + Span: tablepb.Span{TableID: r.Span.TableID}, IsSecondary: true, Checkpoint: r.Checkpoint, }, @@ -1031,8 +1040,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { // Move table, original primary is stopped. msgs, err = r.handleTableStatus(from, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -1053,8 +1062,8 @@ func TestReplicationSetCaptureShutdown(t *testing.T) { // Commit -> Replicating msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1080,9 +1089,10 @@ func TestReplicationSetCaptureShutdownAfterReconstructCommitState(t *testing.T) from := "1" tableID := model.TableID(1) tableStatus := map[model.CaptureID]*tablepb.TableStatus{ - from: {TableID: tableID, State: tablepb.TableStatePrepared}, + from: {Span: tablepb.Span{TableID: tableID}, State: tablepb.TableStatePrepared}, } - r, err := NewReplicationSet(tableID, 0, tableStatus, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, tableStatus, model.ChangeFeedID{}) require.Nil(t, err) require.Equal(t, ReplicationSetStateCommit, r.State) require.Equal(t, "", r.Primary) @@ -1102,7 +1112,8 @@ func TestReplicationSetMoveTableWithHeartbeatResponse(t *testing.T) { t.Parallel() tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, nil, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, nil, model.ChangeFeedID{}) require.Nil(t, err) source := "1" @@ -1121,8 +1132,8 @@ func TestReplicationSetMoveTableWithHeartbeatResponse(t *testing.T) { // Prepare -> Commit. msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -1133,8 +1144,8 @@ func TestReplicationSetMoveTableWithHeartbeatResponse(t *testing.T) { // Source updates it's table status // Source is removed. msgs, err = r.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateStopped, Checkpoint: tablepb.Checkpoint{ CheckpointTs: 3, ResolvedTs: 4, @@ -1152,8 +1163,8 @@ func TestReplicationSetMoveTableWithHeartbeatResponse(t *testing.T) { // Source sends a heartbeat response. msgs, err = r.handleTableStatus(source, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1167,8 +1178,8 @@ func TestReplicationSetMoveTableWithHeartbeatResponse(t *testing.T) { // Commit -> Replicating msgs, err = r.handleTableStatus(dest, &tablepb.TableStatus{ - TableID: tableID, - State: tablepb.TableStateReplicating, + Span: tablepb.Span{TableID: tableID}, + State: tablepb.TableStateReplicating, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1189,7 +1200,8 @@ func TestReplicationSetMoveTableSameDestCapture(t *testing.T) { t.Parallel() tableID := model.TableID(1) - r, err := NewReplicationSet(tableID, 0, nil, model.ChangeFeedID{}) + span := tablepb.Span{TableID: tableID} + r, err := NewReplicationSet(span, 0, nil, model.ChangeFeedID{}) require.Nil(t, err) source := "1" @@ -1221,7 +1233,8 @@ func TestReplicationSetCommitRestart(t *testing.T) { Checkpoint: tablepb.Checkpoint{}, }, } - r, err := NewReplicationSet(0, 0, tableStatus, model.ChangeFeedID{}) + span := tablepb.Span{TableID: 0} + r, err := NewReplicationSet(span, 0, tableStatus, model.ChangeFeedID{}) require.Nil(t, err) require.Equal(t, ReplicationSetStateCommit, r.State) require.EqualValues(t, RoleSecondary, r.Captures["1"]) @@ -1230,8 +1243,8 @@ func TestReplicationSetCommitRestart(t *testing.T) { // Can not promote to primary as there are other captures. msgs, err := r.handleTableStatus("1", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1242,8 +1255,8 @@ func TestReplicationSetCommitRestart(t *testing.T) { // Table status reported by other captures does not change replication set. msgs, err = r.handleTableStatus("2", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStateStopping, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStateStopping, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1255,8 +1268,8 @@ func TestReplicationSetCommitRestart(t *testing.T) { // Only Stopped or Absent allows secondary to be promoted. rClone := clone(r) msgs, err = rClone.handleTableStatus("2", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1265,8 +1278,8 @@ func TestReplicationSetCommitRestart(t *testing.T) { require.Equal(t, "", rClone.Primary) require.NotContains(t, rClone.Captures, "2") msgs, err = r.handleTableStatus("2", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1277,8 +1290,8 @@ func TestReplicationSetCommitRestart(t *testing.T) { // No other captures, promote secondary. msgs, err = r.handleTableStatus("1", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStatePrepared, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStatePrepared, }) require.Nil(t, err) require.Len(t, msgs, 1) @@ -1303,7 +1316,8 @@ func TestReplicationSetRemoveRestart(t *testing.T) { Checkpoint: tablepb.Checkpoint{}, }, } - r, err := NewReplicationSet(0, 0, tableStatus, model.ChangeFeedID{}) + span := tablepb.Span{TableID: 0} + r, err := NewReplicationSet(span, 0, tableStatus, model.ChangeFeedID{}) require.Nil(t, err) require.Equal(t, ReplicationSetStateRemoving, r.State) require.False(t, r.hasRole(RoleSecondary)) @@ -1314,8 +1328,8 @@ func TestReplicationSetRemoveRestart(t *testing.T) { // A capture reports its status. msgs, err := r.handleTableStatus("2", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStateStopping, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStateStopping, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1323,8 +1337,8 @@ func TestReplicationSetRemoveRestart(t *testing.T) { // A capture stopped. msgs, err = r.handleTableStatus("2", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStateStopped, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStateStopped, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1332,8 +1346,8 @@ func TestReplicationSetRemoveRestart(t *testing.T) { // Another capture stopped too. msgs, err = r.handleTableStatus("1", &tablepb.TableStatus{ - TableID: 0, - State: tablepb.TableStateAbsent, + Span: tablepb.Span{TableID: 0}, + State: tablepb.TableStateAbsent, }) require.Nil(t, err) require.Len(t, msgs, 0) @@ -1346,10 +1360,10 @@ func TestReplicationSetHeap_Len(t *testing.T) { h := NewReplicationSetHeap(defaultSlowTableHeapSize) require.Equal(t, 0, h.Len()) - h = append(h, &ReplicationSet{TableID: 0}) + h = append(h, &ReplicationSet{Span: spanz.TableIDToComparableSpan(0)}) require.Equal(t, 1, h.Len()) - h = append(h, &ReplicationSet{TableID: 1}) + h = append(h, &ReplicationSet{Span: spanz.TableIDToComparableSpan(1)}) require.Equal(t, 2, h.Len()) } @@ -1357,9 +1371,18 @@ func TestReplicationSetHeap_Less(t *testing.T) { t.Parallel() h := NewReplicationSetHeap(defaultSlowTableHeapSize) - h = append(h, &ReplicationSet{TableID: 0, Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}}) - h = append(h, &ReplicationSet{TableID: 1, Checkpoint: tablepb.Checkpoint{CheckpointTs: 2, ResolvedTs: 3}}) - h = append(h, &ReplicationSet{TableID: 2, Checkpoint: tablepb.Checkpoint{CheckpointTs: 2, ResolvedTs: 4}}) + h = append(h, &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(0), + Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, + }) + h = append(h, &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(1), + Checkpoint: tablepb.Checkpoint{CheckpointTs: 2, ResolvedTs: 3}, + }) + h = append(h, &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(2), + Checkpoint: tablepb.Checkpoint{CheckpointTs: 2, ResolvedTs: 4}, + }) require.True(t, h.Less(1, 0)) require.True(t, h.Less(2, 1)) } @@ -1369,14 +1392,20 @@ func TestReplicationSetHeap_Basic(t *testing.T) { h := NewReplicationSetHeap(defaultSlowTableHeapSize) heap.Init(&h) - heap.Push(&h, &ReplicationSet{TableID: 0, Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}}) - heap.Push(&h, &ReplicationSet{TableID: 1, Checkpoint: tablepb.Checkpoint{CheckpointTs: 2}}) + heap.Push(&h, &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(0), + Checkpoint: tablepb.Checkpoint{CheckpointTs: 1}, + }) + heap.Push(&h, &ReplicationSet{ + Span: spanz.TableIDToComparableSpan(1), + Checkpoint: tablepb.Checkpoint{CheckpointTs: 2}, + }) require.Equal(t, 2, h.Len()) - require.Equal(t, int64(1), heap.Pop(&h).(*ReplicationSet).TableID) + require.Equal(t, int64(1), heap.Pop(&h).(*ReplicationSet).Span.TableID) require.Equal(t, 1, h.Len()) - require.Equal(t, int64(0), heap.Pop(&h).(*ReplicationSet).TableID) + require.Equal(t, int64(0), heap.Pop(&h).(*ReplicationSet).Span.TableID) require.Equal(t, 0, h.Len()) } @@ -1391,7 +1420,7 @@ func TestReplicationSetHeap_MinK(t *testing.T) { for i := 2 * defaultSlowTableHeapSize; i > 0; i-- { replicationSet := &ReplicationSet{ - TableID: int64(i), + Span: spanz.TableIDToComparableSpan(int64(i)), Checkpoint: tablepb.Checkpoint{CheckpointTs: uint64(i)}, } heap.Push(&h, replicationSet) @@ -1407,12 +1436,12 @@ func TestReplicationSetHeap_MinK(t *testing.T) { expectedTables = append(expectedTables, int64(i)) } - tables := make([]int64, 0) + tables := make([]model.TableID, 0) tableCounts := h.Len() for i := 0; i < tableCounts; i++ { element := heap.Pop(&h).(*ReplicationSet) - fmt.Println(element.TableID) - tables = append(tables, element.TableID) + t.Log(element.Span) + tables = append(tables, element.Span.TableID) } require.Equal(t, expectedTables, tables) require.Equal(t, 0, h.Len()) diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler.go b/cdc/scheduler/internal/v3/scheduler/scheduler.go index 1fad06e60eb..068182d4815 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler.go @@ -15,17 +15,20 @@ package scheduler import ( "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" ) type scheduler interface { Name() string Schedule( checkpointTs model.Ts, - currentTables []model.TableID, + currentSpans []tablepb.Span, aliveCaptures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet) []*replication.ScheduleTask + replications *spanz.Map[*replication.ReplicationSet], + ) []*replication.ScheduleTask } // schedulerPriority is the priority of each scheduler. diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_balance.go b/cdc/scheduler/internal/v3/scheduler/scheduler_balance.go index 86c55cc3745..24cc48bcebb 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_balance.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_balance.go @@ -19,8 +19,10 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" ) var _ scheduler = &balanceScheduler{} @@ -55,9 +57,9 @@ func (b *balanceScheduler) Name() string { func (b *balanceScheduler) Schedule( _ model.Ts, - currentTables []model.TableID, + _ []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], ) []*replication.ScheduleTask { if !b.forceBalance { now := time.Now() @@ -76,29 +78,17 @@ func (b *balanceScheduler) Schedule( } tasks := buildBalanceMoveTables( - b.random, currentTables, captures, replications, b.maxTaskConcurrency) + b.random, captures, replications, b.maxTaskConcurrency) b.forceBalance = len(tasks) != 0 return tasks } func buildBalanceMoveTables( random *rand.Rand, - currentTables []model.TableID, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], maxTaskConcurrency int, ) []*replication.ScheduleTask { - captureTables := make(map[model.CaptureID][]model.TableID) - for _, tableID := range currentTables { - rep, ok := replications[tableID] - if !ok { - continue - } - for captureID := range rep.Captures { - captureTables[captureID] = append(captureTables[captureID], tableID) - } - } - moves := newBalanceMoveTables( random, captures, replications, maxTaskConcurrency, model.ChangeFeedID{}) tasks := make([]*replication.ScheduleTask, 0, len(moves)) diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_balance_test.go b/cdc/scheduler/internal/v3/scheduler/scheduler_balance_test.go index 5f6d78147e0..a8efb07ad35 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_balance_test.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_balance_test.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tiflow/cdc/model" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -31,15 +32,15 @@ func TestSchedulerBalanceCaptureOnline(t *testing.T) { // New capture "b" online captures := map[model.CaptureID]*member.CaptureStatus{"a": {}, "b": {}} - currentTables := []model.TableID{1, 2} - replications := map[model.TableID]*replication.ReplicationSet{ + currentTables := spanz.ArrayToSpan([]model.TableID{1, 2}) + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } + }) tasks := sched.Schedule(0, currentTables, captures, replications) require.Len(t, tasks, 1) require.NotNil(t, tasks[0].MoveTable) - require.Equal(t, tasks[0].MoveTable.TableID, model.TableID(1)) + require.Equal(t, tasks[0].MoveTable.Span.TableID, model.TableID(1)) // New capture "b" online, but this time has capture is stopping captures["a"].State = member.CaptureStateStopping @@ -50,11 +51,11 @@ func TestSchedulerBalanceCaptureOnline(t *testing.T) { // balance interval yet. sched.checkBalanceInterval = time.Hour captures = map[model.CaptureID]*member.CaptureStatus{"a": {}, "b": {}} - currentTables = []model.TableID{1, 2} - replications = map[model.TableID]*replication.ReplicationSet{ + currentTables = spanz.ArrayToSpan([]model.TableID{1, 2}) + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } + }) tasks = sched.Schedule(0, currentTables, captures, replications) require.Len(t, tasks, 1) @@ -62,11 +63,11 @@ func TestSchedulerBalanceCaptureOnline(t *testing.T) { sched.checkBalanceInterval = time.Hour sched.forceBalance = false captures = map[model.CaptureID]*member.CaptureStatus{"a": {}, "b": {}} - currentTables = []model.TableID{1, 2} - replications = map[model.TableID]*replication.ReplicationSet{ + currentTables = spanz.ArrayToSpan([]model.TableID{1, 2}) + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } + }) tasks = sched.Schedule(0, currentTables, captures, replications) require.Len(t, tasks, 0) } @@ -79,13 +80,13 @@ func TestSchedulerBalanceTaskLimit(t *testing.T) { // New capture "b" online captures := map[model.CaptureID]*member.CaptureStatus{"a": {}, "b": {}} - currentTables := []model.TableID{1, 2, 3, 4} - replications := map[model.TableID]*replication.ReplicationSet{ + currentTables := spanz.ArrayToSpan([]model.TableID{1, 2, 3, 4}) + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 3: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 4: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } + }) tasks := sched.Schedule(0, currentTables, captures, replications) require.Len(t, tasks, 2) diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_basic.go b/cdc/scheduler/internal/v3/scheduler/scheduler_basic.go index 4e80ac3dced..35518459fc7 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_basic.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_basic.go @@ -19,8 +19,10 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -53,33 +55,33 @@ func (b *basicScheduler) Name() string { func (b *basicScheduler) Schedule( checkpointTs model.Ts, - currentTables []model.TableID, + currentSpans []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], ) []*replication.ScheduleTask { tasks := make([]*replication.ScheduleTask, 0) - tablesLenEqual := len(currentTables) == len(replications) + tablesLenEqual := len(currentSpans) == replications.Len() tablesAllFind := true - newTables := make([]model.TableID, 0) - for _, tableID := range currentTables { - if len(newTables) >= b.batchSize { + newSpans := make([]tablepb.Span, 0) + for _, span := range currentSpans { + if len(newSpans) >= b.batchSize { break } - rep, ok := replications[tableID] + rep, ok := replications.Get(span) if !ok { - newTables = append(newTables, tableID) + newSpans = append(newSpans, span) // The table ID is not in the replication means the two sets are // not identical. tablesAllFind = false continue } if rep.State == replication.ReplicationSetStateAbsent { - newTables = append(newTables, tableID) + newSpans = append(newSpans, span) } } // Build add table tasks. - if len(newTables) > 0 { + if len(newSpans) > 0 { captureIDs := make([]model.CaptureID, 0, len(captures)) for captureID, status := range captures { if status.State == member.CaptureStateStopping { @@ -108,9 +110,9 @@ func (b *basicScheduler) Schedule( zap.String("namespace", b.changefeedID.Namespace), zap.String("changefeed", b.changefeedID.ID), zap.Strings("captureIDs", captureIDs), - zap.Int64s("tableIDs", newTables)) + zap.Int("tableCount", len(newSpans))) tasks = append( - tasks, newBurstAddTables(checkpointTs, newTables, captureIDs)) + tasks, newBurstAddTables(checkpointTs, newSpans, captureIDs)) } // Build remove table tasks. @@ -121,24 +123,29 @@ func (b *basicScheduler) Schedule( // and for all tables in currentTables have a record in replications. if !tablesLenEqual || !tablesAllFind { // The two sets are not identical. We need to find removed tables. - intersectionTable := make(map[model.TableID]struct{}, len(currentTables)) - for _, tableID := range currentTables { - _, ok := replications[tableID] + intersectionTable := spanz.NewMap[struct{}]() + for _, span := range currentSpans { + _, ok := replications.Get(span) if !ok { continue } - intersectionTable[tableID] = struct{}{} + intersectionTable.ReplaceOrInsert(span, struct{}{}) } - rmTables := make([]model.TableID, 0) - for tableID := range replications { - _, ok := intersectionTable[tableID] + rmSpans := make([]tablepb.Span, 0) + replications.Ascend(func(span tablepb.Span, value *replication.ReplicationSet) bool { + ok := intersectionTable.Has(span) if !ok { - rmTables = append(rmTables, tableID) + rmSpans = append(rmSpans, span) } - } - if len(rmTables) > 0 { + return true + }) + if len(rmSpans) > 0 { + log.Info("schedulerv3: burst remove table", + zap.String("namespace", b.changefeedID.Namespace), + zap.String("changefeed", b.changefeedID.ID), + zap.Int("tableCount", len(newSpans))) tasks = append(tasks, - newBurstRemoveTables(rmTables, replications, b.changefeedID)) + newBurstRemoveTables(rmSpans, replications, b.changefeedID)) } } return tasks @@ -146,13 +153,13 @@ func (b *basicScheduler) Schedule( // newBurstAddTables add each new table to captures in a round-robin way. func newBurstAddTables( - checkpointTs model.Ts, newTables []model.TableID, captureIDs []model.CaptureID, + checkpointTs model.Ts, newSpans []tablepb.Span, captureIDs []model.CaptureID, ) *replication.ScheduleTask { idx := 0 - tables := make([]replication.AddTable, 0, len(newTables)) - for _, tableID := range newTables { + tables := make([]replication.AddTable, 0, len(newSpans)) + for _, span := range newSpans { tables = append(tables, replication.AddTable{ - TableID: tableID, + Span: span, CaptureID: captureIDs[idx], CheckpointTs: checkpointTs, }) @@ -167,12 +174,12 @@ func newBurstAddTables( } func newBurstRemoveTables( - rmTables []model.TableID, replications map[model.TableID]*replication.ReplicationSet, + rmSpans []tablepb.Span, replications *spanz.Map[*replication.ReplicationSet], changefeedID model.ChangeFeedID, ) *replication.ScheduleTask { - tables := make([]replication.RemoveTable, 0, len(rmTables)) - for _, tableID := range rmTables { - rep := replications[tableID] + tables := make([]replication.RemoveTable, 0, len(rmSpans)) + for _, span := range rmSpans { + rep := replications.GetV(span) var captureID model.CaptureID for id := range rep.Captures { captureID = id @@ -186,7 +193,7 @@ func newBurstRemoveTables( continue } tables = append(tables, replication.RemoveTable{ - TableID: tableID, + Span: span, CaptureID: captureID, }) } diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_basic_test.go b/cdc/scheduler/internal/v3/scheduler/scheduler_basic_test.go index 25ec37cd3a5..491d1847cda 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_basic_test.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_basic_test.go @@ -18,20 +18,30 @@ import ( "testing" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) +func mapToSpanMap[T any](in map[model.TableID]T) *spanz.Map[T] { + out := spanz.NewMap[T]() + for tableID, v := range in { + out.ReplaceOrInsert(tablepb.Span{TableID: tableID}, v) + } + return out +} + func TestSchedulerBasic(t *testing.T) { t.Parallel() captures := map[model.CaptureID]*member.CaptureStatus{"a": {}, "b": {}} - currentTables := []model.TableID{1, 2, 3, 4} + currentTables := spanz.ArrayToSpan([]model.TableID{1, 2, 3, 4}) // Initial table dispatch. // AddTable only - replications := map[model.TableID]*replication.ReplicationSet{} + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{}) b := newBasicScheduler(2, model.ChangeFeedID{}) // one capture stopping, another one is initialized @@ -52,12 +62,12 @@ func TestSchedulerBasic(t *testing.T) { tasks = b.Schedule(0, currentTables, captures, replications) require.Len(t, tasks, 1) require.Len(t, tasks[0].BurstBalance.AddTables, 2) - require.Equal(t, tasks[0].BurstBalance.AddTables[0].TableID, model.TableID(1)) - require.Equal(t, tasks[0].BurstBalance.AddTables[1].TableID, model.TableID(2)) + require.Equal(t, tasks[0].BurstBalance.AddTables[0].Span.TableID, model.TableID(1)) + require.Equal(t, tasks[0].BurstBalance.AddTables[1].Span.TableID, model.TableID(2)) // Capture offline, causes replication.ReplicationSetStateAbsent. // AddTable only. - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: { State: replication.ReplicationSetStateReplicating, Primary: "a", Captures: map[string]replication.Role{ @@ -77,15 +87,15 @@ func TestSchedulerBasic(t *testing.T) { }, }, 4: {State: replication.ReplicationSetStateAbsent}, - } + }) tasks = b.Schedule(1, currentTables, captures, replications) require.Len(t, tasks, 1) - require.Equal(t, tasks[0].BurstBalance.AddTables[0].TableID, model.TableID(4)) + require.Equal(t, tasks[0].BurstBalance.AddTables[0].Span.TableID, model.TableID(4)) require.Equal(t, tasks[0].BurstBalance.AddTables[0].CheckpointTs, model.Ts(1)) // DDL CREATE/DROP/TRUNCATE TABLE. // AddTable 4, and RemoveTable 5. - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: { State: replication.ReplicationSetStateReplicating, Primary: "a", Captures: map[string]replication.Role{ @@ -110,21 +120,21 @@ func TestSchedulerBasic(t *testing.T) { "a": replication.RoleUndetermined, "b": replication.RoleSecondary, }, }, - } + }) tasks = b.Schedule(2, currentTables, captures, replications) require.Len(t, tasks, 2) if tasks[0].BurstBalance.AddTables != nil { - require.Equal(t, tasks[0].BurstBalance.AddTables[0].TableID, model.TableID(4)) + require.Equal(t, tasks[0].BurstBalance.AddTables[0].Span.TableID, model.TableID(4)) require.Equal(t, tasks[0].BurstBalance.AddTables[0].CheckpointTs, model.Ts(2)) - require.Equal(t, tasks[1].BurstBalance.RemoveTables[0].TableID, model.TableID(5)) + require.Equal(t, tasks[1].BurstBalance.RemoveTables[0].Span.TableID, model.TableID(5)) } else { - require.Equal(t, tasks[1].BurstBalance.AddTables[0].TableID, model.TableID(4)) + require.Equal(t, tasks[1].BurstBalance.AddTables[0].Span.TableID, model.TableID(4)) require.Equal(t, tasks[0].BurstBalance.AddTables[0].CheckpointTs, model.Ts(2)) - require.Equal(t, tasks[0].BurstBalance.RemoveTables[0].TableID, model.TableID(5)) + require.Equal(t, tasks[0].BurstBalance.RemoveTables[0].Span.TableID, model.TableID(5)) } // RemoveTable only. - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: { State: replication.ReplicationSetStateReplicating, Primary: "a", Captures: map[string]replication.Role{ @@ -155,10 +165,10 @@ func TestSchedulerBasic(t *testing.T) { "b": replication.RoleUndetermined, }, }, - } + }) tasks = b.Schedule(3, currentTables, captures, replications) require.Len(t, tasks, 1) - require.Equal(t, tasks[0].BurstBalance.RemoveTables[0].TableID, model.TableID(5)) + require.Equal(t, tasks[0].BurstBalance.RemoveTables[0].Span.TableID, model.TableID(5)) } func TestSchedulerPriority(t *testing.T) { @@ -176,9 +186,9 @@ func benchmarkSchedulerBalance( b *testing.B, factory func(total int) ( name string, - currentTables []model.TableID, + currentTables []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], sched scheduler, ), ) { @@ -198,9 +208,9 @@ func benchmarkSchedulerBalance( func BenchmarkSchedulerBasicAddTables(b *testing.B) { benchmarkSchedulerBalance(b, func(total int) ( name string, - currentTables []model.TableID, + currentTables []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], sched scheduler, ) { const captureCount = 8 @@ -208,11 +218,11 @@ func BenchmarkSchedulerBasicAddTables(b *testing.B) { for i := 0; i < captureCount; i++ { captures[fmt.Sprint(i)] = &member.CaptureStatus{} } - currentTables = make([]model.TableID, 0, total) + currentTables = make([]tablepb.Span, 0, total) for i := 0; i < total; i++ { - currentTables = append(currentTables, int64(10000+i)) + currentTables = append(currentTables, tablepb.Span{TableID: int64(10000 + i)}) } - replications = map[model.TableID]*replication.ReplicationSet{} + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{}) name = fmt.Sprintf("AddTable %d", total) sched = newBasicScheduler(50, model.ChangeFeedID{}) return name, currentTables, captures, replications, sched @@ -222,9 +232,9 @@ func BenchmarkSchedulerBasicAddTables(b *testing.B) { func BenchmarkSchedulerBasicRemoveTables(b *testing.B) { benchmarkSchedulerBalance(b, func(total int) ( name string, - currentTables []model.TableID, + currentTables []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], sched scheduler, ) { const captureCount = 8 @@ -232,12 +242,13 @@ func BenchmarkSchedulerBasicRemoveTables(b *testing.B) { for i := 0; i < captureCount; i++ { captures[fmt.Sprint(i)] = &member.CaptureStatus{} } - currentTables = make([]model.TableID, 0, total) - replications = map[model.TableID]*replication.ReplicationSet{} + currentTables = make([]tablepb.Span, 0, total) + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{}) for i := 0; i < total; i++ { - replications[int64(10000+i)] = &replication.ReplicationSet{ - Primary: fmt.Sprint(i % captureCount), - } + replications.ReplaceOrInsert(tablepb.Span{TableID: int64(10000 + i)}, + &replication.ReplicationSet{ + Primary: fmt.Sprint(i % captureCount), + }) } name = fmt.Sprintf("RemoveTable %d", total) sched = newBasicScheduler(50, model.ChangeFeedID{}) @@ -248,9 +259,9 @@ func BenchmarkSchedulerBasicRemoveTables(b *testing.B) { func BenchmarkSchedulerBasicAddRemoveTables(b *testing.B) { benchmarkSchedulerBalance(b, func(total int) ( name string, - currentTables []model.TableID, + currentTables []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], sched scheduler, ) { const captureCount = 8 @@ -258,15 +269,16 @@ func BenchmarkSchedulerBasicAddRemoveTables(b *testing.B) { for i := 0; i < captureCount; i++ { captures[fmt.Sprint(i)] = &member.CaptureStatus{} } - currentTables = make([]model.TableID, 0, total) + currentTables = make([]tablepb.Span, 0, total) for i := 0; i < total/2; i++ { - currentTables = append(currentTables, int64(100000+i)) + currentTables = append(currentTables, tablepb.Span{TableID: int64(100000 + i)}) } - replications = map[model.TableID]*replication.ReplicationSet{} + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{}) for i := 0; i < total/2; i++ { - replications[int64(200000+i)] = &replication.ReplicationSet{ - Primary: fmt.Sprint(i % captureCount), - } + replications.ReplaceOrInsert(tablepb.Span{TableID: int64(200000 + i)}, + &replication.ReplicationSet{ + Primary: fmt.Sprint(i % captureCount), + }) } name = fmt.Sprintf("AddRemoveTable %d", total) sched = newBasicScheduler(50, model.ChangeFeedID{}) diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture.go b/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture.go index b98c3f8c5ed..e14533b01fe 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture.go @@ -19,8 +19,10 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -70,9 +72,9 @@ func (d *drainCaptureScheduler) setTarget(target model.CaptureID) bool { func (d *drainCaptureScheduler) Schedule( _ model.Ts, - _ []model.TableID, + _ []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], ) []*replication.ScheduleTask { d.mu.Lock() defer d.mu.Unlock() @@ -123,9 +125,10 @@ func (d *drainCaptureScheduler) Schedule( } maxTaskConcurrency := d.maxTaskConcurrency - // victimTables record tables should be moved out from the target capture - victimTables := make([]model.TableID, 0, maxTaskConcurrency) - for tableID, rep := range replications { + // victimSpans record tables should be moved out from the target capture + victimSpans := make([]tablepb.Span, 0, maxTaskConcurrency) + skipDrain := false + replications.Ascend(func(span tablepb.Span, rep *replication.ReplicationSet) bool { if rep.State != replication.ReplicationSetStateReplicating { // only drain the target capture if all tables is replicating, log.Debug("schedulerv3: drain capture scheduler skip this tick,"+ @@ -134,12 +137,13 @@ func (d *drainCaptureScheduler) Schedule( zap.String("changefeed", d.changefeedID.ID), zap.String("target", d.target), zap.Any("replication", rep)) - return nil + skipDrain = true + return false } if rep.Primary == d.target { - if len(victimTables) < maxTaskConcurrency { - victimTables = append(victimTables, tableID) + if len(victimSpans) < maxTaskConcurrency { + victimSpans = append(victimSpans, span) } } @@ -147,13 +151,17 @@ func (d *drainCaptureScheduler) Schedule( if rep.Primary != d.target { captureWorkload[rep.Primary]++ } + return true + }) + if skipDrain { + return nil } // this always indicate that the whole draining process finished, and can be triggered by: // 1. the target capture has no table at the beginning // 2. all tables moved from the target capture // 3. the target capture cannot be found in the latest captures - if len(victimTables) == 0 { + if len(victimSpans) == 0 { log.Info("schedulerv3: drain capture scheduler finished, since no table", zap.String("namespace", d.changefeedID.Namespace), zap.String("changefeed", d.changefeedID.ID), @@ -164,7 +172,7 @@ func (d *drainCaptureScheduler) Schedule( // For each victim table, find the target for it result := make([]*replication.ScheduleTask, 0, maxTaskConcurrency) - for _, tableID := range victimTables { + for _, span := range victimSpans { target := "" minWorkload := math.MaxInt64 for captureID, workload := range captureWorkload { @@ -183,7 +191,7 @@ func (d *drainCaptureScheduler) Schedule( result = append(result, &replication.ScheduleTask{ MoveTable: &replication.MoveTable{ - TableID: tableID, + Span: span, DestCapture: target, }, Accept: (replication.Callback)(nil), // No need for accept callback here. diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture_test.go b/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture_test.go index c9efa2c472d..03f1a99197e 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture_test.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_drain_capture_test.go @@ -17,8 +17,10 @@ import ( "testing" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -30,8 +32,8 @@ func TestDrainCapture(t *testing.T) { var checkpointTs model.Ts captures := make(map[model.CaptureID]*member.CaptureStatus) - currentTables := make([]model.TableID, 0) - replications := make(map[model.TableID]*replication.ReplicationSet) + currentTables := make([]tablepb.Span, 0) + replications := mapToSpanMap(make(map[model.TableID]*replication.ReplicationSet)) tasks := scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) @@ -54,8 +56,8 @@ func TestDrainCapture(t *testing.T) { require.Equal(t, captureIDNotDraining, scheduler.target) captures["b"] = &member.CaptureStatus{} - currentTables = []model.TableID{1, 2, 3, 4, 5, 6, 7} - replications = map[model.TableID]*replication.ReplicationSet{ + currentTables = spanz.ArrayToSpan([]model.TableID{1, 2, 3, 4, 5, 6, 7}) + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: { State: replication.ReplicationSetStateReplicating, Primary: "a", Captures: map[string]replication.Role{ @@ -98,7 +100,7 @@ func TestDrainCapture(t *testing.T) { "b": replication.RoleSecondary, }, }, - } + }) ok = scheduler.setTarget("a") require.True(t, ok) @@ -107,14 +109,14 @@ func TestDrainCapture(t *testing.T) { require.Equal(t, "a", scheduler.target) require.Len(t, tasks, 0) - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 3: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 4: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, 6: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, 7: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, - } + }) tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Equal(t, "a", scheduler.target) @@ -132,8 +134,8 @@ func TestDrainStoppingCapture(t *testing.T) { var checkpointTs model.Ts captures := make(map[model.CaptureID]*member.CaptureStatus) - currentTables := make([]model.TableID, 0) - replications := make(map[model.TableID]*replication.ReplicationSet) + currentTables := make([]tablepb.Span, 0) + replications := mapToSpanMap(make(map[model.TableID]*replication.ReplicationSet)) scheduler := newDrainCaptureScheduler(10, model.ChangeFeedID{}) tasks := scheduler.Schedule(checkpointTs, currentTables, captures, replications) @@ -141,13 +143,13 @@ func TestDrainStoppingCapture(t *testing.T) { captures["a"] = &member.CaptureStatus{} captures["b"] = &member.CaptureStatus{State: member.CaptureStateStopping} - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, - } + }) tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 1) - require.EqualValues(t, 2, tasks[0].MoveTable.TableID) + require.EqualValues(t, 2, tasks[0].MoveTable.Span.TableID) require.EqualValues(t, "a", tasks[0].MoveTable.DestCapture) require.EqualValues(t, "b", scheduler.getTarget()) } @@ -156,15 +158,15 @@ func TestDrainSkipOwner(t *testing.T) { t.Parallel() var checkpointTs model.Ts - currentTables := make([]model.TableID, 0) + currentTables := make([]tablepb.Span, 0) captures := map[model.CaptureID]*member.CaptureStatus{ "a": {}, "b": {IsOwner: true, State: member.CaptureStateStopping}, } - replications := map[model.TableID]*replication.ReplicationSet{ + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, - } + }) scheduler := newDrainCaptureScheduler(10, model.ChangeFeedID{}) tasks := scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) @@ -175,15 +177,15 @@ func TestDrainImbalanceCluster(t *testing.T) { t.Parallel() var checkpointTs model.Ts - currentTables := make([]model.TableID, 0) + currentTables := make([]tablepb.Span, 0) captures := map[model.CaptureID]*member.CaptureStatus{ "a": {State: member.CaptureStateInitialized}, "b": {IsOwner: true, State: member.CaptureStateInitialized}, } - replications := map[model.TableID]*replication.ReplicationSet{ + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } + }) scheduler := newDrainCaptureScheduler(10, model.ChangeFeedID{}) scheduler.setTarget("a") tasks := scheduler.Schedule(checkpointTs, currentTables, captures, replications) @@ -195,18 +197,18 @@ func TestDrainEvenlyDistributedTables(t *testing.T) { t.Parallel() var checkpointTs model.Ts - currentTables := make([]model.TableID, 0) + currentTables := make([]tablepb.Span, 0) captures := map[model.CaptureID]*member.CaptureStatus{ "a": {State: member.CaptureStateInitialized}, "b": {IsOwner: true, State: member.CaptureStateInitialized}, "c": {State: member.CaptureStateInitialized}, } - replications := map[model.TableID]*replication.ReplicationSet{ + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 3: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 6: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, - } + }) scheduler := newDrainCaptureScheduler(10, model.ChangeFeedID{}) scheduler.setTarget("a") tasks := scheduler.Schedule(checkpointTs, currentTables, captures, replications) diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_manager.go b/cdc/scheduler/internal/v3/scheduler/scheduler_manager.go index 7673f7148ce..9c6f47b7691 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_manager.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_manager.go @@ -19,9 +19,11 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -63,22 +65,22 @@ func NewSchedulerManager( // Schedule generates schedule tasks based on the inputs. func (sm *Manager) Schedule( checkpointTs model.Ts, - currentTables []model.TableID, + currentSpans []tablepb.Span, aliveCaptures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, - runTasking map[model.TableID]*replication.ScheduleTask, + replications *spanz.Map[*replication.ReplicationSet], + runTasking *spanz.Map[*replication.ScheduleTask], ) []*replication.ScheduleTask { for sid, scheduler := range sm.schedulers { // Basic scheduler bypasses max task check, because it handles the most // critical scheduling, e.g. add table via CREATE TABLE DDL. if sid != int(schedulerPriorityBasic) { - if len(runTasking) >= sm.maxTaskConcurrency { + if runTasking.Len() >= sm.maxTaskConcurrency { // Do not generate more scheduling tasks if there are too many // running tasks. return nil } } - tasks := scheduler.Schedule(checkpointTs, currentTables, aliveCaptures, replications) + tasks := scheduler.Schedule(checkpointTs, currentSpans, aliveCaptures, replications) for _, t := range tasks { name := struct { scheduler, task string @@ -99,7 +101,7 @@ func (sm *Manager) Schedule( } // MoveTable moves a table to the target capture. -func (sm *Manager) MoveTable(tableID model.TableID, target model.CaptureID) { +func (sm *Manager) MoveTable(span tablepb.Span, target model.CaptureID) { scheduler := sm.schedulers[schedulerPriorityMoveTable] moveTableScheduler, ok := scheduler.(*moveTableScheduler) if !ok { @@ -107,12 +109,12 @@ func (sm *Manager) MoveTable(tableID model.TableID, target model.CaptureID) { zap.String("namespace", sm.changefeedID.Namespace), zap.String("changefeed", sm.changefeedID.ID)) } - if !moveTableScheduler.addTask(tableID, target) { + if !moveTableScheduler.addTask(span, target) { log.Info("schedulerv3: manual move Table task ignored, "+ "since the last triggered task not finished", zap.String("namespace", sm.changefeedID.Namespace), zap.String("changefeed", sm.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("targetCapture", target)) } } diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_manager_test.go b/cdc/scheduler/internal/v3/scheduler/scheduler_manager_test.go index dd291ccb848..2a5a887b7e7 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_manager_test.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_manager_test.go @@ -17,9 +17,11 @@ import ( "testing" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" "github.com/pingcap/tiflow/pkg/config" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -47,35 +49,34 @@ func TestSchedulerManagerScheduler(t *testing.T) { "a": {State: member.CaptureStateInitialized}, "b": {State: member.CaptureStateInitialized}, } - currentTables := []model.TableID{1} - + currentSpans := []tablepb.Span{{TableID: 1}} // schedulerPriorityBasic bypasses task check. - replications := map[model.TableID]*replication.ReplicationSet{} - runningTasks := map[model.TableID]*replication.ScheduleTask{1: {}} - tasks := m.Schedule(0, currentTables, captures, replications, runningTasks) + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{}) + runningTasks := mapToSpanMap(map[model.TableID]*replication.ScheduleTask{1: {}}) + tasks := m.Schedule(0, currentSpans, captures, replications, runningTasks) require.Len(t, tasks, 1) // No more task. - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } - tasks = m.Schedule(0, currentTables, captures, replications, runningTasks) + }) + tasks = m.Schedule(0, currentSpans, captures, replications, runningTasks) require.Len(t, tasks, 0) - // Move table is drop because of running tasks. - m.MoveTable(1, "b") - replications = map[model.TableID]*replication.ReplicationSet{ + // Move table is dropped because of running tasks. + m.MoveTable(tablepb.Span{TableID: 1}, "b") + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } - tasks = m.Schedule(0, currentTables, captures, replications, runningTasks) + }) + tasks = m.Schedule(0, currentSpans, captures, replications, runningTasks) require.Len(t, tasks, 0) // Move table can proceed after clean up tasks. - m.MoveTable(1, "b") - replications = map[model.TableID]*replication.ReplicationSet{ + m.MoveTable(tablepb.Span{TableID: 1}, "b") + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } - runningTasks = map[model.TableID]*replication.ScheduleTask{} - tasks = m.Schedule(0, currentTables, captures, replications, runningTasks) + }) + runningTasks = spanz.NewMap[*replication.ScheduleTask]() + tasks = m.Schedule(0, currentSpans, captures, replications, runningTasks) require.Len(t, tasks, 1) } diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_move_table.go b/cdc/scheduler/internal/v3/scheduler/scheduler_move_table.go index d8019e99eb9..b2b3c7f1a14 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_move_table.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_move_table.go @@ -18,8 +18,10 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -27,14 +29,14 @@ var _ scheduler = &moveTableScheduler{} type moveTableScheduler struct { mu sync.Mutex - tasks map[model.TableID]*replication.ScheduleTask + tasks *spanz.Map[*replication.ScheduleTask] changefeedID model.ChangeFeedID } func newMoveTableScheduler(changefeed model.ChangeFeedID) *moveTableScheduler { return &moveTableScheduler{ - tasks: make(map[model.TableID]*replication.ScheduleTask), + tasks: spanz.NewMap[*replication.ScheduleTask](), changefeedID: changefeed, } } @@ -43,39 +45,42 @@ func (m *moveTableScheduler) Name() string { return "move-table-scheduler" } -func (m *moveTableScheduler) addTask(tableID model.TableID, target model.CaptureID) bool { +func (m *moveTableScheduler) addTask(span tablepb.Span, target model.CaptureID) bool { // previous triggered task not accepted yet, decline the new manual move table request. m.mu.Lock() defer m.mu.Unlock() - if _, ok := m.tasks[tableID]; ok { + if ok := m.tasks.Has(span); ok { return false } - m.tasks[tableID] = &replication.ScheduleTask{ + m.tasks.ReplaceOrInsert(span, &replication.ScheduleTask{ MoveTable: &replication.MoveTable{ - TableID: tableID, + Span: span, DestCapture: target, }, Accept: func() { m.mu.Lock() defer m.mu.Unlock() - delete(m.tasks, tableID) + m.tasks.Delete(span) }, - } + }) return true } func (m *moveTableScheduler) Schedule( _ model.Ts, - currentTables []model.TableID, + currentSpans []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], ) []*replication.ScheduleTask { m.mu.Lock() defer m.mu.Unlock() + // FIXME: moveTableScheduler is broken in the sense of range level replication. + // It is impossible for users to pass valid start key and end key. + result := make([]*replication.ScheduleTask, 0) - if len(m.tasks) == 0 { + if m.tasks.Len() == 0 { return result } @@ -83,22 +88,23 @@ func (m *moveTableScheduler) Schedule( return result } - allTables := model.NewTableSet() - for _, tableID := range currentTables { - allTables.Add(tableID) + allSpans := spanz.NewSet() + for _, span := range currentSpans { + allSpans.Add(span) } - for tableID, task := range m.tasks { + toBeDeleted := []tablepb.Span{} + m.tasks.Ascend(func(span tablepb.Span, task *replication.ScheduleTask) bool { // table may not in the all current tables // if it was removed after manual move table triggered. - if !allTables.Contain(tableID) { + if !allSpans.Contain(span) { log.Warn("schedulerv3: move table ignored, since the table cannot found", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("captureID", task.MoveTable.DestCapture)) - delete(m.tasks, tableID) - continue + toBeDeleted = append(toBeDeleted, span) + return true } // the target capture may offline after manual move table triggered. @@ -107,47 +113,52 @@ func (m *moveTableScheduler) Schedule( log.Info("schedulerv3: move table ignored, since the target capture cannot found", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("captureID", task.MoveTable.DestCapture)) - delete(m.tasks, tableID) - continue + toBeDeleted = append(toBeDeleted, span) + return true } if status.State != member.CaptureStateInitialized { log.Warn("schedulerv3: move table ignored, target capture is not initialized", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("captureID", task.MoveTable.DestCapture), zap.Any("state", status.State)) - delete(m.tasks, tableID) - continue + toBeDeleted = append(toBeDeleted, span) + return true } - rep, ok := replications[tableID] + rep, ok := replications.Get(span) if !ok { log.Warn("schedulerv3: move table ignored, table not found in the replication set", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("captureID", task.MoveTable.DestCapture)) - delete(m.tasks, tableID) - continue + toBeDeleted = append(toBeDeleted, span) + return true } // only move replicating table. if rep.State != replication.ReplicationSetStateReplicating { log.Info("schedulerv3: move table ignored, since the table is not replicating now", zap.String("namespace", m.changefeedID.Namespace), zap.String("changefeed", m.changefeedID.ID), - zap.Int64("tableID", tableID), + zap.Stringer("span", &span), zap.String("captureID", task.MoveTable.DestCapture), zap.Any("replicationState", rep.State)) - delete(m.tasks, tableID) + toBeDeleted = append(toBeDeleted, span) } + return true + }) + for _, span := range toBeDeleted { + m.tasks.Delete(span) } - for _, task := range m.tasks { - result = append(result, task) - } + m.tasks.Ascend(func(span tablepb.Span, value *replication.ScheduleTask) bool { + result = append(result, value) + return true + }) return result } diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_move_table_test.go b/cdc/scheduler/internal/v3/scheduler/scheduler_move_table_test.go index 63e33f1d36f..f01ed67d05f 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_move_table_test.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_move_table_test.go @@ -17,8 +17,10 @@ import ( "testing" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -31,11 +33,11 @@ func TestSchedulerMoveTable(t *testing.T) { }, "b": { State: member.CaptureStateInitialized, }} - currentTables := []model.TableID{1, 2, 3, 4} + currentTables := spanz.ArrayToSpan([]model.TableID{1, 2, 3, 4}) - replications := map[model.TableID]*replication.ReplicationSet{ + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, - } + }) scheduler := newMoveTableScheduler(model.ChangeFeedID{}) require.Equal(t, "move-table-scheduler", scheduler.Name()) @@ -44,44 +46,44 @@ func TestSchedulerMoveTable(t *testing.T) { checkpointTs, currentTables, map[model.CaptureID]*member.CaptureStatus{}, replications) require.Len(t, tasks, 0) - scheduler.addTask(model.TableID(0), "a") + scheduler.addTask(tablepb.Span{TableID: 0}, "a") tasks = scheduler.Schedule( checkpointTs, currentTables, map[model.CaptureID]*member.CaptureStatus{}, replications) require.Len(t, tasks, 0) // move a not exist table - scheduler.addTask(model.TableID(0), "a") + scheduler.addTask(tablepb.Span{TableID: 0}, "a") tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) // move table to a not exist capture - scheduler.addTask(model.TableID(1), "c") + scheduler.addTask(tablepb.Span{TableID: 1}, "c") tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) // move table not replicating - scheduler.addTask(model.TableID(1), "b") + scheduler.addTask(tablepb.Span{TableID: 1}, "b") tasks = scheduler.Schedule( - checkpointTs, currentTables, captures, map[model.TableID]*replication.ReplicationSet{}) + checkpointTs, currentTables, captures, spanz.NewMap[*replication.ReplicationSet]()) require.Len(t, tasks, 0) - scheduler.addTask(model.TableID(1), "b") - replications[model.TableID(1)].State = replication.ReplicationSetStatePrepare + scheduler.addTask(tablepb.Span{TableID: 1}, "b") + replications.GetV(tablepb.Span{TableID: 1}).State = replication.ReplicationSetStatePrepare tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) - scheduler.addTask(model.TableID(1), "b") - replications[model.TableID(1)].State = replication.ReplicationSetStateReplicating + scheduler.addTask(tablepb.Span{TableID: 1}, "b") + replications.GetV(tablepb.Span{TableID: 1}).State = replication.ReplicationSetStateReplicating tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 1) - require.Equal(t, model.TableID(1), tasks[0].MoveTable.TableID) + require.Equal(t, model.TableID(1), tasks[0].MoveTable.Span.TableID) require.Equal(t, "b", tasks[0].MoveTable.DestCapture) - require.Equal(t, scheduler.tasks[model.TableID(1)], tasks[0]) + require.Equal(t, scheduler.tasks.GetV(tablepb.Span{TableID: 1}), tasks[0]) // the target capture is stopping - scheduler.addTask(model.TableID(1), "b") + scheduler.addTask(tablepb.Span{TableID: 1}, "b") captures["b"].State = member.CaptureStateStopping tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) - require.NotContains(t, scheduler.tasks, model.TableID(1)) + require.False(t, scheduler.tasks.Has(tablepb.Span{TableID: 1})) } diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance.go b/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance.go index 94e36360a31..152a51c1207 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance.go @@ -22,8 +22,10 @@ import ( "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "go.uber.org/zap" ) @@ -50,9 +52,9 @@ func (r *rebalanceScheduler) Name() string { func (r *rebalanceScheduler) Schedule( _ model.Ts, - currentTables []model.TableID, + currentSpans []tablepb.Span, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], ) []*replication.ScheduleTask { // rebalance is not triggered, or there is still some pending task, // do not generate new tasks. @@ -75,8 +77,8 @@ func (r *rebalanceScheduler) Schedule( } // only rebalance when all tables are replicating - for _, tableID := range currentTables { - rep, ok := replications[tableID] + for _, span := range currentSpans { + rep, ok := replications.Get(span) if !ok { return nil } @@ -108,28 +110,28 @@ func (r *rebalanceScheduler) Schedule( func newBalanceMoveTables( random *rand.Rand, captures map[model.CaptureID]*member.CaptureStatus, - replications map[model.TableID]*replication.ReplicationSet, + replications *spanz.Map[*replication.ReplicationSet], maxTaskLimit int, changefeedID model.ChangeFeedID, ) []replication.MoveTable { - tablesPerCapture := make(map[model.CaptureID]*model.TableSet) + tablesPerCapture := make(map[model.CaptureID]*spanz.Set) for captureID := range captures { - tablesPerCapture[captureID] = model.NewTableSet() + tablesPerCapture[captureID] = spanz.NewSet() } - for tableID, rep := range replications { - if rep.State != replication.ReplicationSetStateReplicating { - continue + replications.Ascend(func(span tablepb.Span, rep *replication.ReplicationSet) bool { + if rep.State == replication.ReplicationSetStateReplicating { + tablesPerCapture[rep.Primary].Add(span) } - tablesPerCapture[rep.Primary].Add(tableID) - } + return true + }) // findVictim return tables which need to be moved - upperLimitPerCapture := int(math.Ceil(float64(len(replications)) / float64(len(captures)))) + upperLimitPerCapture := int(math.Ceil(float64(replications.Len()) / float64(len(captures)))) - victims := make([]model.TableID, 0) + victims := make([]tablepb.Span, 0) for _, ts := range tablesPerCapture { - tables := ts.Keys() + spans := ts.Keys() if random != nil { // Complexity note: Shuffle has O(n), where `n` is the number of tables. // Also, during a single call of `Schedule`, Shuffle can be called at most @@ -137,28 +139,28 @@ func newBalanceMoveTables( // Only called when a rebalance is triggered, which happens rarely, // we do not expect a performance degradation as a result of adding // the randomness. - random.Shuffle(len(tables), func(i, j int) { - tables[i], tables[j] = tables[j], tables[i] + random.Shuffle(len(spans), func(i, j int) { + spans[i], spans[j] = spans[j], spans[i] }) } else { - // sort the tableIDs here so that the result is deterministic, + // sort the spans here so that the result is deterministic, // which would aid testing and debugging. - sort.Slice(tables, func(i, j int) bool { - return tables[i] < tables[j] + sort.Slice(spans, func(i, j int) bool { + return spans[i].Less(&spans[j]) }) } - tableNum2Remove := len(tables) - upperLimitPerCapture + tableNum2Remove := len(spans) - upperLimitPerCapture if tableNum2Remove <= 0 { continue } - for _, table := range tables { + for _, span := range spans { if tableNum2Remove <= 0 { break } - victims = append(victims, table) - ts.Remove(table) + victims = append(victims, span) + ts.Remove(span) tableNum2Remove-- } } @@ -172,7 +174,7 @@ func newBalanceMoveTables( } // for each victim table, find the target for it moveTables := make([]replication.MoveTable, 0, len(victims)) - for idx, tableID := range victims { + for idx, span := range victims { target := "" minWorkload := math.MaxInt64 @@ -195,10 +197,10 @@ func newBalanceMoveTables( } moveTables = append(moveTables, replication.MoveTable{ - TableID: tableID, + Span: span, DestCapture: target, }) - tablesPerCapture[target].Add(tableID) + tablesPerCapture[target].Add(span) captureWorkload[target] = randomizeWorkload(random, tablesPerCapture[target].Size()) } diff --git a/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance_test.go b/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance_test.go index 88ba54eeb1d..e6b1ff53d83 100644 --- a/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance_test.go +++ b/cdc/scheduler/internal/v3/scheduler/scheduler_rebalance_test.go @@ -18,8 +18,10 @@ import ( "testing" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/processor/tablepb" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/member" "github.com/pingcap/tiflow/cdc/scheduler/internal/v3/replication" + "github.com/pingcap/tiflow/pkg/spanz" "github.com/stretchr/testify/require" ) @@ -28,9 +30,9 @@ func TestSchedulerRebalance(t *testing.T) { var checkpointTs model.Ts captures := map[model.CaptureID]*member.CaptureStatus{"a": {}, "b": {}} - currentTables := []model.TableID{1, 2, 3, 4} + currentTables := spanz.ArrayToSpan([]model.TableID{1, 2, 3, 4}) - replications := map[model.TableID]*replication.ReplicationSet{ + replications := mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: { State: replication.ReplicationSetStateReplicating, Primary: "a", Captures: map[string]replication.Role{ @@ -50,7 +52,7 @@ func TestSchedulerRebalance(t *testing.T) { }, }, 4: {State: replication.ReplicationSetStateAbsent}, - } + }) scheduler := newRebalanceScheduler(model.ChangeFeedID{}) require.Equal(t, "rebalance-scheduler", scheduler.Name()) @@ -65,7 +67,8 @@ func TestSchedulerRebalance(t *testing.T) { require.Len(t, tasks, 0) // table not in the replication set, - tasks = scheduler.Schedule(checkpointTs, []model.TableID{0}, captures, replications) + tasks = scheduler.Schedule( + checkpointTs, spanz.ArrayToSpan([]model.TableID{0}), captures, replications) require.Len(t, tasks, 0) // not all tables are replicating, @@ -73,24 +76,24 @@ func TestSchedulerRebalance(t *testing.T) { require.Len(t, tasks, 0) // table distribution is balanced, should have no task. - replications = map[model.TableID]*replication.ReplicationSet{ + replications = mapToSpanMap(map[model.TableID]*replication.ReplicationSet{ 1: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 2: {State: replication.ReplicationSetStateReplicating, Primary: "a"}, 3: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, 4: {State: replication.ReplicationSetStateReplicating, Primary: "b"}, - } + }) tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 0) // Imbalance. - replications[5] = &replication.ReplicationSet{ + replications.ReplaceOrInsert(tablepb.Span{TableID: 5}, &replication.ReplicationSet{ State: replication.ReplicationSetStateReplicating, Primary: "a", - } - replications[6] = &replication.ReplicationSet{ + }) + replications.ReplaceOrInsert(tablepb.Span{TableID: 6}, &replication.ReplicationSet{ State: replication.ReplicationSetStateReplicating, Primary: "a", - } + }) // capture is stopping, ignore the request captures["a"].State = member.CaptureStateStopping @@ -104,7 +107,7 @@ func TestSchedulerRebalance(t *testing.T) { tasks = scheduler.Schedule(checkpointTs, currentTables, captures, replications) require.Len(t, tasks, 1) require.Contains(t, tasks[0].BurstBalance.MoveTables, replication.MoveTable{ - TableID: 1, DestCapture: "b", + Span: tablepb.Span{TableID: 1}, DestCapture: "b", }) require.EqualValues(t, 1, atomic.LoadInt32(&scheduler.rebalance)) diff --git a/cdc/scheduler/rexport.go b/cdc/scheduler/rexport.go index 6a6f1b18f95..26e40ae088d 100644 --- a/cdc/scheduler/rexport.go +++ b/cdc/scheduler/rexport.go @@ -25,6 +25,7 @@ import ( "github.com/pingcap/tiflow/pkg/p2p" "github.com/pingcap/tiflow/pkg/pdutil" "github.com/prometheus/client_golang/prometheus" + "github.com/tikv/client-go/v2/tikv" ) // TableExecutor is an abstraction for "Processor". @@ -84,12 +85,13 @@ func NewScheduler( messageServer *p2p.MessageServer, messageRouter p2p.MessageRouter, ownerRevision int64, - cfg *config.SchedulerConfig, + regionCache *tikv.RegionCache, pdClock pdutil.Clock, + cfg *config.SchedulerConfig, ) (Scheduler, error) { return v3.NewCoordinator( ctx, captureID, changeFeedID, - messageServer, messageRouter, ownerRevision, cfg, pdClock) + messageServer, messageRouter, ownerRevision, regionCache, pdClock, cfg) } // InitMetrics registers all metrics used in scheduler From fcb961f83585ec1ec01ff37f5d64de0a1d55e4fd Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Wed, 14 Dec 2022 10:40:42 +0800 Subject: [PATCH 2/5] address lints Signed-off-by: Neil Shen --- .../internal/v3/replication/replication_manager.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/cdc/scheduler/internal/v3/replication/replication_manager.go b/cdc/scheduler/internal/v3/replication/replication_manager.go index ef3fb320035..1a776613f30 100644 --- a/cdc/scheduler/internal/v3/replication/replication_manager.go +++ b/cdc/scheduler/internal/v3/replication/replication_manager.go @@ -148,9 +148,9 @@ func (r *Manager) HandleCaptureChanges( } var err error spanStatusMap.Ascend(func(span tablepb.Span, status map[string]*tablepb.TableStatus) bool { - table, err := NewReplicationSet(span, checkpointTs, status, r.changefeedID) + table, err1 := NewReplicationSet(span, checkpointTs, status, r.changefeedID) if err != nil { - err = errors.Trace(err) + err = errors.Trace(err1) return false } r.spans.ReplaceOrInsert(table.Span, table) @@ -165,9 +165,9 @@ func (r *Manager) HandleCaptureChanges( var err error r.spans.Ascend(func(span tablepb.Span, table *ReplicationSet) bool { for captureID := range removed { - msgs, affected, err := table.handleCaptureShutdown(captureID) + msgs, affected, err1 := table.handleCaptureShutdown(captureID) if err != nil { - err = errors.Trace(err) + err = errors.Trace(err1) return false } sentMsgs = append(sentMsgs, msgs...) From c439b71fdc807456a2859cc24e5c1e3407544e95 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Thu, 24 Nov 2022 19:52:59 +0800 Subject: [PATCH 3/5] ticdc: disable span replication in mysql/tidb changefeed Signed-off-by: Neil Shen --- cdc/owner/changefeed.go | 26 +++++++++----- cdc/owner/changefeed_test.go | 2 +- cdc/owner/owner_test.go | 6 ++-- cdc/processor/manager.go | 14 +++++++- cdc/processor/manager_test.go | 3 +- cdc/processor/processor.go | 3 ++ cdc/processor/processor_test.go | 7 ++-- cdc/sink/validator.go | 9 +++++ cdc/sink/validator_test.go | 64 +++++++++++++++++++++++++++++++++ 9 files changed, 119 insertions(+), 15 deletions(-) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index aa15323c933..1cf3301406f 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -27,6 +27,7 @@ import ( "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/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" @@ -42,24 +43,23 @@ import ( // newSchedulerFromCtx creates a new scheduler from context. // This function is factored out to facilitate unit testing. func newSchedulerFromCtx( - ctx cdcContext.Context, up *upstream.Upstream, + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, ) (ret scheduler.Scheduler, err error) { changeFeedID := ctx.ChangefeedVars().ID messageServer := ctx.GlobalVars().MessageServer messageRouter := ctx.GlobalVars().MessageRouter ownerRev := ctx.GlobalVars().OwnerRevision captureID := ctx.GlobalVars().CaptureInfo.ID - cfg := config.GetGlobalServerConfig().Debug ret, err = scheduler.NewScheduler( ctx, captureID, changeFeedID, - messageServer, messageRouter, ownerRev, up.RegionCache, up.PDClock, cfg.Scheduler) + messageServer, messageRouter, ownerRev, up.RegionCache, up.PDClock, cfg) return ret, errors.Trace(err) } func newScheduler( - ctx cdcContext.Context, up *upstream.Upstream, + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, ) (scheduler.Scheduler, error) { - return newSchedulerFromCtx(ctx, up) + return newSchedulerFromCtx(ctx, up, cfg) } type changefeed struct { @@ -68,6 +68,7 @@ type changefeed struct { state *orchestrator.ChangefeedReactorState upstream *upstream.Upstream + cfg *config.SchedulerConfig scheduler scheduler.Scheduler // barriers will be created when a changefeed is initialized // and will be destroyed when a changefeed is closed. @@ -126,7 +127,9 @@ type changefeed struct { ) (puller.DDLPuller, error) newSink func(changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, reportErr func(error)) DDLSink - newScheduler func(ctx cdcContext.Context, up *upstream.Upstream) (scheduler.Scheduler, error) + newScheduler func( + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, + ) (scheduler.Scheduler, error) lastDDLTs uint64 // Timestamp of the last executed DDL. Only used for tests. } @@ -164,7 +167,9 @@ func newChangefeed4Test( changefeed model.ChangeFeedID, ) (puller.DDLPuller, error), newSink func(changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, reportErr func(err error)) DDLSink, - newScheduler func(ctx cdcContext.Context, up *upstream.Upstream) (scheduler.Scheduler, error), + newScheduler func( + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, + ) (scheduler.Scheduler, error), ) *changefeed { c := newChangefeed(id, state, up) c.newDDLPuller = newDDLPuller @@ -539,7 +544,12 @@ LOOP: zap.String("changefeed", c.id.ID)) // create scheduler - c.scheduler, err = c.newScheduler(ctx, c.upstream) + // TODO: Remove the hack once span replication is compatible with all sinks. + cfg := *c.cfg + if !sink.IsSinkCompatibleWithSpanReplication(c.state.Info.SinkURI) { + cfg.RegionPerSpan = 0 + } + c.scheduler, err = c.newScheduler(ctx, c.upstream, &cfg) if err != nil { return errors.Trace(err) } diff --git a/cdc/owner/changefeed_test.go b/cdc/owner/changefeed_test.go index 391475837ef..9955b5badd6 100644 --- a/cdc/owner/changefeed_test.go +++ b/cdc/owner/changefeed_test.go @@ -215,7 +215,7 @@ func createChangefeed4Test(ctx cdcContext.Context, t *testing.T, }, // new scheduler func( - ctx cdcContext.Context, up *upstream.Upstream, + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, ) (scheduler.Scheduler, error) { return &mockScheduler{}, nil }) diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index 02cb0eb7d98..42495a1617f 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -62,7 +62,9 @@ func newOwner4Test( changefeed model.ChangeFeedID, ) (puller.DDLPuller, error), newSink func(changefeedID model.ChangeFeedID, info *model.ChangeFeedInfo, reportErr func(err error)) DDLSink, - newScheduler func(ctx cdcContext.Context, up *upstream.Upstream) (scheduler.Scheduler, error), + newScheduler func( + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, + ) (scheduler.Scheduler, error), pdClient pd.Client, ) Owner { m := upstream.NewManager4Test(pdClient) @@ -102,7 +104,7 @@ func createOwner4Test(ctx cdcContext.Context, t *testing.T) (*ownerImpl, *orches }, // new scheduler func( - ctx cdcContext.Context, up *upstream.Upstream, + ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, ) (scheduler.Scheduler, error) { return &mockScheduler{}, nil }, diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 84b8ac42a06..089132fdcef 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -23,6 +23,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/log" "github.com/pingcap/tiflow/cdc/model" + "github.com/pingcap/tiflow/cdc/sink" + "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" cerrors "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" @@ -71,7 +73,9 @@ type managerImpl struct { model.ChangeFeedID, *upstream.Upstream, *model.Liveness, + *config.SchedulerConfig, ) *processor + cfg *config.SchedulerConfig metricProcessorCloseDuration prometheus.Observer } @@ -118,7 +122,15 @@ func (m *managerImpl) Tick(stdCtx context.Context, state orchestrator.ReactorSta up = m.upstreamManager.AddUpstream(upstreamInfo) } failpoint.Inject("processorManagerHandleNewChangefeedDelay", nil) - p = m.newProcessor(changefeedState, m.captureInfo, changefeedID, up, m.liveness) + + // TODO: Remove the hack once span replication is compatible with + // all sinks. + cfg := *m.cfg + if !sink.IsSinkCompatibleWithSpanReplication(changefeedState.Info.SinkURI) { + cfg.RegionPerSpan = 0 + } + p = m.newProcessor( + changefeedState, m.captureInfo, changefeedID, up, m.liveness, &cfg) m.processors[changefeedID] = p } ctx := cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index e39e6fecba0..a8cac164646 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -57,8 +57,9 @@ func NewManager4Test( changefeedID model.ChangeFeedID, up *upstream.Upstream, liveness *model.Liveness, + cfg *config.SchedulerConfig, ) *processor { - return newProcessor4Test(t, state, captureInfo, createTablePipeline, m.liveness) + return newProcessor4Test(t, state, captureInfo, createTablePipeline, m.liveness, cfg) } return m } diff --git a/cdc/processor/processor.go b/cdc/processor/processor.go index e0ae90a349e..38cccee1223 100644 --- a/cdc/processor/processor.go +++ b/cdc/processor/processor.go @@ -94,6 +94,7 @@ type processor struct { ctx cdcContext.Context, span tablepb.Span, replicaInfo *model.TableReplicaInfo, ) (tablepb.TablePipeline, error) newAgent func(cdcContext.Context, *model.Liveness) (scheduler.Agent, error) + cfg *config.SchedulerConfig liveness *model.Liveness agent scheduler.Agent @@ -537,6 +538,7 @@ func newProcessor( changefeedID model.ChangeFeedID, up *upstream.Upstream, liveness *model.Liveness, + cfg *config.SchedulerConfig, ) *processor { p := &processor{ changefeed: state, @@ -580,6 +582,7 @@ func newProcessor( p.createTablePipeline = p.createTablePipelineImpl p.lazyInit = p.lazyInitImpl p.newAgent = p.newAgentImpl + p.cfg = cfg return p } diff --git a/cdc/processor/processor_test.go b/cdc/processor/processor_test.go index 033105ed888..6b0777ec79a 100644 --- a/cdc/processor/processor_test.go +++ b/cdc/processor/processor_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tiflow/cdc/redo" "github.com/pingcap/tiflow/cdc/scheduler" mocksink "github.com/pingcap/tiflow/cdc/sink/mock" + "github.com/pingcap/tiflow/pkg/config" cdcContext "github.com/pingcap/tiflow/pkg/context" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/etcd" @@ -48,12 +49,13 @@ func newProcessor4Test( ctx cdcContext.Context, span tablepb.Span, replicaInfo *model.TableReplicaInfo, ) (tablepb.TablePipeline, error), liveness *model.Liveness, + cfg *config.SchedulerConfig, ) *processor { up := upstream.NewUpstream4Test(nil) p := newProcessor( state, captureInfo, - model.ChangeFeedID4Test("processor-test", "processor-test"), up, liveness) + model.ChangeFeedID4Test("processor-test", "processor-test"), up, liveness, cfg) p.lazyInit = func(ctx cdcContext.Context) error { p.agent = &mockAgent{executor: p} p.sinkV1 = mocksink.NewNormalMockSink() @@ -106,7 +108,8 @@ func initProcessor4Test( changefeed := orchestrator.NewChangefeedReactorState( etcd.DefaultCDCClusterID, ctx.ChangefeedVars().ID) captureInfo := &model.CaptureInfo{ID: "capture-test", AdvertiseAddr: "127.0.0.1:0000"} - p := newProcessor4Test(t, changefeed, captureInfo, newMockTablePipeline, liveness) + cfg := config.NewDefaultSchedulerConfig() + p := newProcessor4Test(t, changefeed, captureInfo, newMockTablePipeline, liveness, cfg) captureID := ctx.GlobalVars().CaptureInfo.ID changefeedID := ctx.ChangefeedVars().ID diff --git a/cdc/sink/validator.go b/cdc/sink/validator.go index b0a4142febe..e05e0772dd7 100644 --- a/cdc/sink/validator.go +++ b/cdc/sink/validator.go @@ -16,6 +16,7 @@ package sink import ( "context" "net/url" + "strings" "github.com/pingcap/tiflow/cdc/contextutil" "github.com/pingcap/tiflow/cdc/model" @@ -143,3 +144,11 @@ func checkBDRMode(ctx context.Context, sinkURI *url.URL, replicaConfig *config.R } return nil } + +// IsSinkCompatibleWithSpanReplication returns true if the sink uri is +// compatible with span replication. +func IsSinkCompatibleWithSpanReplication(sinkURI string) bool { + u, err := url.Parse(sinkURI) + return err == nil && + (strings.Contains(u.Scheme, "kafka") || strings.Contains(u.Scheme, "blackhole")) +} diff --git a/cdc/sink/validator_test.go b/cdc/sink/validator_test.go index a176d21b557..d101acd259b 100644 --- a/cdc/sink/validator_test.go +++ b/cdc/sink/validator_test.go @@ -87,3 +87,67 @@ func TestPreCheckSinkURI(t *testing.T) { }) } } + +func TestIsSinkCompatibleWithSpanReplication(t *testing.T) { + t.Parallel() + + tests := []struct { + name string + uri string + compatible bool + }{ + { + name: "MySQL URI", + uri: "mysql://root:111@foo.bar:3306/", + compatible: false, + }, + { + name: "TiDB URI", + uri: "tidb://root:111@foo.bar:3306/", + compatible: false, + }, + { + name: "MySQL URI", + uri: "mysql+ssl://root:111@foo.bar:3306/", + compatible: false, + }, + { + name: "TiDB URI", + uri: "tidb+ssl://root:111@foo.bar:3306/", + compatible: false, + }, + { + name: "Kafka URI", + uri: "kafka://foo.bar:3306/topic", + compatible: true, + }, + { + name: "Kafka URI", + uri: "kafka+ssl://foo.bar:3306/topic", + compatible: true, + }, + { + name: "Blackhole URI", + uri: "blackhole://foo.bar:3306/topic", + compatible: true, + }, + { + name: "Unknown URI", + uri: "unknown://foo.bar:3306", + compatible: false, + }, + { + name: "Error URI", + uri: "error/foo.bar:3306", + compatible: false, + }, + } + + for _, tt := range tests { + test := tt + t.Run(test.name, func(t *testing.T) { + compatible := IsSinkCompatibleWithSpanReplication(test.uri) + require.Equal(t, compatible, tt.compatible) + }) + } +} From 4e97f1a0bd7862e9d5dbd997b43c86da66517984 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Wed, 14 Dec 2022 18:07:11 +0800 Subject: [PATCH 4/5] address lint Signed-off-by: Neil Shen --- cdc/sink/validator_test.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/cdc/sink/validator_test.go b/cdc/sink/validator_test.go index d101acd259b..ed722a7f458 100644 --- a/cdc/sink/validator_test.go +++ b/cdc/sink/validator_test.go @@ -144,10 +144,7 @@ func TestIsSinkCompatibleWithSpanReplication(t *testing.T) { } for _, tt := range tests { - test := tt - t.Run(test.name, func(t *testing.T) { - compatible := IsSinkCompatibleWithSpanReplication(test.uri) - require.Equal(t, compatible, tt.compatible) - }) + compatible := IsSinkCompatibleWithSpanReplication(tt.uri) + require.Equal(t, compatible, tt.compatible, tt.name) } } From 595113903fe76ed9f61e05239b485d7aa992b779 Mon Sep 17 00:00:00 2001 From: Neil Shen Date: Wed, 14 Dec 2022 22:23:36 +0800 Subject: [PATCH 5/5] fix nil pointer Signed-off-by: Neil Shen --- cdc/capture/capture.go | 8 +++++--- cdc/owner/changefeed.go | 5 ++++- cdc/owner/owner.go | 11 +++++++++-- cdc/owner/owner_test.go | 5 +++-- cdc/processor/manager.go | 2 ++ cdc/processor/manager_test.go | 9 ++++++--- 6 files changed, 29 insertions(+), 11 deletions(-) diff --git a/cdc/capture/capture.go b/cdc/capture/capture.go index 0d5eab8ac02..70af8acbf98 100644 --- a/cdc/capture/capture.go +++ b/cdc/capture/capture.go @@ -126,8 +126,9 @@ type captureImpl struct { captureInfo *model.CaptureInfo, upstreamManager *upstream.Manager, liveness *model.Liveness, + cfg *config.SchedulerConfig, ) processor.Manager - newOwner func(upstreamManager *upstream.Manager) owner.Owner + newOwner func(upstreamManager *upstream.Manager, cfg *config.SchedulerConfig) owner.Owner } // NewCapture returns a new Capture instance @@ -239,7 +240,8 @@ func (c *captureImpl) reset(ctx context.Context) error { return cerror.WrapError(cerror.ErrNewCaptureFailed, err) } - c.processorManager = c.newProcessorManager(c.info, c.upstreamManager, &c.liveness) + c.processorManager = c.newProcessorManager( + c.info, c.upstreamManager, &c.liveness, c.config.Debug.Scheduler) if c.session != nil { // It can't be handled even after it fails, so we ignore it. _ = c.session.Close() @@ -463,7 +465,7 @@ func (c *captureImpl) campaignOwner(ctx cdcContext.Context) error { zap.String("captureID", c.info.ID), zap.Int64("ownerRev", ownerRev)) - owner := c.newOwner(c.upstreamManager) + owner := c.newOwner(c.upstreamManager, c.config.Debug.Scheduler) c.setOwner(owner) globalState := orchestrator.NewGlobalState(c.EtcdClient.GetClusterID()) diff --git a/cdc/owner/changefeed.go b/cdc/owner/changefeed.go index 1cf3301406f..f37a2b916e8 100644 --- a/cdc/owner/changefeed.go +++ b/cdc/owner/changefeed.go @@ -138,6 +138,7 @@ func newChangefeed( id model.ChangeFeedID, state *orchestrator.ChangefeedReactorState, up *upstream.Upstream, + cfg *config.SchedulerConfig, ) *changefeed { c := &changefeed{ id: id, @@ -155,6 +156,7 @@ func newChangefeed( newSink: newDDLSink, } c.newScheduler = newScheduler + c.cfg = cfg return c } @@ -171,7 +173,8 @@ func newChangefeed4Test( ctx cdcContext.Context, up *upstream.Upstream, cfg *config.SchedulerConfig, ) (scheduler.Scheduler, error), ) *changefeed { - c := newChangefeed(id, state, up) + cfg := config.NewDefaultSchedulerConfig() + c := newChangefeed(id, state, up, cfg) c.newDDLPuller = newDDLPuller c.newSink = newSink c.newScheduler = newScheduler diff --git a/cdc/owner/owner.go b/cdc/owner/owner.go index e56cc956628..8211eaeaeef 100644 --- a/cdc/owner/owner.go +++ b/cdc/owner/owner.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/log" "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" cerror "github.com/pingcap/tiflow/pkg/errors" "github.com/pingcap/tiflow/pkg/orchestrator" @@ -132,7 +133,9 @@ type ownerImpl struct { id model.ChangeFeedID, state *orchestrator.ChangefeedReactorState, up *upstream.Upstream, + cfg *config.SchedulerConfig, ) *changefeed + cfg *config.SchedulerConfig // removedChangefeed is a workload of https://github.com/pingcap/tiflow/issues/7657 // by delaying recreate changefeed with the same ID. @@ -142,7 +145,10 @@ type ownerImpl struct { } // NewOwner creates a new Owner -func NewOwner(upstreamManager *upstream.Manager) Owner { +func NewOwner( + upstreamManager *upstream.Manager, + cfg *config.SchedulerConfig, +) Owner { return &ownerImpl{ upstreamManager: upstreamManager, changefeeds: make(map[model.ChangeFeedID]*changefeed), @@ -151,6 +157,7 @@ func NewOwner(upstreamManager *upstream.Manager) Owner { logLimiter: rate.NewLimiter(versionInconsistentLogRate, versionInconsistentLogRate), removedChangefeed: make(map[model.ChangeFeedID]time.Time), removedSinkURI: make(map[url.URL]time.Time), + cfg: cfg, } } @@ -207,7 +214,7 @@ func (o *ownerImpl) Tick(stdCtx context.Context, rawState orchestrator.ReactorSt upstreamInfo := state.Upstreams[changefeedState.Info.UpstreamID] up = o.upstreamManager.AddUpstream(upstreamInfo) } - cfReactor = o.newChangefeed(changefeedID, changefeedState, up) + cfReactor = o.newChangefeed(changefeedID, changefeedState, up, o.cfg) o.changefeeds[changefeedID] = cfReactor } ctx = cdcContext.WithChangefeedVars(ctx, &cdcContext.ChangefeedVars{ diff --git a/cdc/owner/owner_test.go b/cdc/owner/owner_test.go index 42495a1617f..749c099ebb3 100644 --- a/cdc/owner/owner_test.go +++ b/cdc/owner/owner_test.go @@ -68,13 +68,14 @@ func newOwner4Test( pdClient pd.Client, ) Owner { m := upstream.NewManager4Test(pdClient) - o := NewOwner(m).(*ownerImpl) + o := NewOwner(m, config.NewDefaultSchedulerConfig()).(*ownerImpl) // Most tests do not need to test bootstrap. o.bootstrapped = true o.newChangefeed = func( id model.ChangeFeedID, state *orchestrator.ChangefeedReactorState, up *upstream.Upstream, + cfg *config.SchedulerConfig, ) *changefeed { return newChangefeed4Test(id, state, up, newDDLPuller, newSink, newScheduler) } @@ -419,7 +420,7 @@ func TestAdminJob(t *testing.T) { func TestUpdateGCSafePoint(t *testing.T) { mockPDClient := &gc.MockPDClient{} m := upstream.NewManager4Test(mockPDClient) - o := NewOwner(m).(*ownerImpl) + o := NewOwner(m, config.NewDefaultSchedulerConfig()).(*ownerImpl) ctx := cdcContext.NewBackendContext4Test(true) ctx, cancel := cdcContext.WithCancel(ctx) defer cancel() diff --git a/cdc/processor/manager.go b/cdc/processor/manager.go index 089132fdcef..fa89d55b453 100644 --- a/cdc/processor/manager.go +++ b/cdc/processor/manager.go @@ -85,6 +85,7 @@ func NewManager( captureInfo *model.CaptureInfo, upstreamManager *upstream.Manager, liveness *model.Liveness, + cfg *config.SchedulerConfig, ) Manager { return &managerImpl{ captureInfo: captureInfo, @@ -94,6 +95,7 @@ func NewManager( upstreamManager: upstreamManager, newProcessor: newProcessor, metricProcessorCloseDuration: processorCloseDuration, + cfg: cfg, } } diff --git a/cdc/processor/manager_test.go b/cdc/processor/manager_test.go index a8cac164646..6940998810d 100644 --- a/cdc/processor/manager_test.go +++ b/cdc/processor/manager_test.go @@ -50,7 +50,8 @@ func NewManager4Test( liveness *model.Liveness, ) *managerImpl { captureInfo := &model.CaptureInfo{ID: "capture-test", AdvertiseAddr: "127.0.0.1:0000"} - m := NewManager(captureInfo, upstream.NewManager4Test(nil), liveness).(*managerImpl) + cfg := config.NewDefaultSchedulerConfig() + m := NewManager(captureInfo, upstream.NewManager4Test(nil), liveness, cfg).(*managerImpl) m.newProcessor = func( state *orchestrator.ChangefeedReactorState, captureInfo *model.CaptureInfo, @@ -237,7 +238,8 @@ func TestClose(t *testing.T) { func TestSendCommandError(t *testing.T) { liveness := model.LivenessCaptureAlive - m := NewManager(&model.CaptureInfo{ID: "capture-test"}, nil, &liveness).(*managerImpl) + cfg := config.NewDefaultSchedulerConfig() + m := NewManager(&model.CaptureInfo{ID: "capture-test"}, nil, &liveness, cfg).(*managerImpl) ctx, cancel := context.WithCancel(context.TODO()) cancel() // Use unbuffered channel to stable test. @@ -299,7 +301,8 @@ func TestManagerLiveness(t *testing.T) { func TestQueryTableCount(t *testing.T) { liveness := model.LivenessCaptureAlive - m := NewManager(&model.CaptureInfo{ID: "capture-test"}, nil, &liveness).(*managerImpl) + cfg := config.NewDefaultSchedulerConfig() + m := NewManager(&model.CaptureInfo{ID: "capture-test"}, nil, &liveness, cfg).(*managerImpl) ctx := context.TODO() // Add some tables to processor. tables := spanz.NewMap[tablepb.TablePipeline]()