From 64430c359ef5e8ce8f246ab64c7bd3ee7cbd52e1 Mon Sep 17 00:00:00 2001 From: okJiang <819421878@qq.com> Date: Wed, 13 Dec 2023 11:33:22 +0800 Subject: [PATCH] disttask: rename dispatcher to scheduler (#49182) close pingcap/tidb#49125 --- Makefile | 4 +- pkg/ddl/BUILD.bazel | 8 +- pkg/ddl/backfilling_clean_s3.go | 8 +- ...tcher.go => backfilling_dist_scheduler.go} | 92 ++-- ....go => backfilling_dist_scheduler_test.go} | 54 +-- pkg/ddl/ddl.go | 10 +- pkg/ddl/index.go | 6 +- pkg/ddl/ingest/tests/partition_table_test.go | 2 +- pkg/disttask/framework/BUILD.bazel | 2 +- .../framework_dynamic_dispatch_test.go | 12 +- .../framework/framework_err_handling_test.go | 6 +- pkg/disttask/framework/framework_ha_test.go | 12 +- .../framework_pause_and_resume_test.go | 30 +- .../framework/framework_rollback_test.go | 6 +- pkg/disttask/framework/framework_test.go | 64 +-- pkg/disttask/framework/handle/handle_test.go | 4 +- pkg/disttask/framework/mock/BUILD.bazel | 2 +- .../{dispatcher_mock.go => scheduler_mock.go} | 44 +- pkg/disttask/framework/proto/task.go | 6 +- .../{dispatcher => scheduler}/BUILD.bazel | 18 +- .../{dispatcher => scheduler}/interface.go | 70 +-- .../{dispatcher => scheduler}/main_test.go | 16 +- .../mock/BUILD.bazel | 6 +- .../mock/scheduler_mock.go} | 10 +- .../rebalance_test.go | 26 +- .../dispatcher.go => scheduler/scheduler.go} | 450 +++++++++--------- .../scheduler_manager.go} | 240 +++++----- .../scheduler_manager_test.go} | 20 +- .../scheduler_test.go} | 166 +++---- .../{dispatcher => scheduler}/slots.go | 10 +- .../{dispatcher => scheduler}/slots_test.go | 2 +- .../state_transform.go | 2 +- pkg/disttask/framework/storage/table_test.go | 2 +- pkg/disttask/framework/storage/task_table.go | 23 +- .../framework/taskexecutor/BUILD.bazel | 2 +- .../framework/taskexecutor/manager.go | 4 +- .../framework/taskexecutor/task_executor.go | 18 +- pkg/disttask/framework/testutil/BUILD.bazel | 6 +- pkg/disttask/framework/testutil/context.go | 2 +- .../framework/testutil/dispatcher_util.go | 298 ------------ .../framework/testutil/disttest_util.go | 40 +- .../framework/testutil/scheduler_util.go | 298 ++++++++++++ pkg/disttask/importinto/BUILD.bazel | 10 +- pkg/disttask/importinto/clean_s3.go | 10 +- pkg/disttask/importinto/metrics.go | 2 +- pkg/disttask/importinto/proto.go | 6 +- .../{dispatcher.go => scheduler.go} | 186 ++++---- .../{dispatcher_test.go => scheduler_test.go} | 52 +- ...tkit_test.go => scheduler_testkit_test.go} | 24 +- pkg/disttask/importinto/task_executor.go | 2 +- pkg/domain/BUILD.bazel | 2 +- pkg/domain/domain.go | 20 +- pkg/metrics/disttask.go | 14 +- tests/realtikvtest/addindextest1/BUILD.bazel | 2 +- .../addindextest1/disttask_test.go | 8 +- tests/realtikvtest/addindextest2/BUILD.bazel | 2 +- .../addindextest2/global_sort_test.go | 10 +- tests/realtikvtest/importintotest/BUILD.bazel | 2 +- .../importintotest/import_into_test.go | 16 +- .../realtikvtest/importintotest4/BUILD.bazel | 2 +- .../importintotest4/global_sort_test.go | 10 +- 61 files changed, 1241 insertions(+), 1240 deletions(-) rename pkg/ddl/{backfilling_dispatcher.go => backfilling_dist_scheduler.go} (84%) rename pkg/ddl/{backfilling_dispatcher_test.go => backfilling_dist_scheduler_test.go} (87%) rename pkg/disttask/framework/mock/{dispatcher_mock.go => scheduler_mock.go} (94%) rename pkg/disttask/framework/{dispatcher => scheduler}/BUILD.bazel (86%) rename pkg/disttask/framework/{dispatcher => scheduler}/interface.go (77%) rename pkg/disttask/framework/{dispatcher => scheduler}/main_test.go (81%) rename pkg/disttask/framework/{dispatcher => scheduler}/mock/BUILD.bazel (76%) rename pkg/disttask/framework/{dispatcher/mock/dispatcher_mock.go => scheduler/mock/scheduler_mock.go} (91%) rename pkg/disttask/framework/{dispatcher => scheduler}/rebalance_test.go (96%) rename pkg/disttask/framework/{dispatcher/dispatcher.go => scheduler/scheduler.go} (55%) rename pkg/disttask/framework/{dispatcher/dispatcher_manager.go => scheduler/scheduler_manager.go} (56%) rename pkg/disttask/framework/{dispatcher/dispatcher_manager_test.go => scheduler/scheduler_manager_test.go} (87%) rename pkg/disttask/framework/{dispatcher/dispatcher_test.go => scheduler/scheduler_test.go} (74%) rename pkg/disttask/framework/{dispatcher => scheduler}/slots.go (95%) rename pkg/disttask/framework/{dispatcher => scheduler}/slots_test.go (99%) rename pkg/disttask/framework/{dispatcher => scheduler}/state_transform.go (99%) delete mode 100644 pkg/disttask/framework/testutil/dispatcher_util.go create mode 100644 pkg/disttask/framework/testutil/scheduler_util.go rename pkg/disttask/importinto/{dispatcher.go => scheduler.go} (76%) rename pkg/disttask/importinto/{dispatcher_test.go => scheduler_test.go} (78%) rename pkg/disttask/importinto/{dispatcher_testkit_test.go => scheduler_testkit_test.go} (94%) diff --git a/Makefile b/Makefile index 46d3b4d4d0fe8..9230fae1be0bf 100644 --- a/Makefile +++ b/Makefile @@ -387,8 +387,8 @@ mock_lightning: tools/bin/mockgen gen_mock: tools/bin/mockgen tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor TaskTable,Pool,TaskExecutor,Extension > pkg/disttask/framework/mock/task_executor_mock.go - tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/dispatcher Dispatcher,CleanUpRoutine,TaskManager > pkg/disttask/framework/mock/dispatcher_mock.go - tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/dispatcher Extension > pkg/disttask/framework/dispatcher/mock/dispatcher_mock.go + tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/scheduler Scheduler,CleanUpRoutine,TaskManager > pkg/disttask/framework/mock/scheduler_mock.go + tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/scheduler Extension > pkg/disttask/framework/scheduler/mock/scheduler_mock.go tools/bin/mockgen -package execute github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/execute SubtaskExecutor > pkg/disttask/framework/mock/execute/execute_mock.go tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/importinto MiniTaskExecutor > pkg/disttask/importinto/mock/import_mock.go tools/bin/mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/planner LogicalPlan,PipelineSpec > pkg/disttask/framework/mock/plan_mock.go diff --git a/pkg/ddl/BUILD.bazel b/pkg/ddl/BUILD.bazel index 655da1e38fd9e..fb397d00080fe 100644 --- a/pkg/ddl/BUILD.bazel +++ b/pkg/ddl/BUILD.bazel @@ -13,8 +13,8 @@ go_library( srcs = [ "backfilling.go", "backfilling_clean_s3.go", - "backfilling_dispatcher.go", "backfilling_dist_executor.go", + "backfilling_dist_scheduler.go", "backfilling_import_cloud.go", "backfilling_merge_sort.go", "backfilling_operators.go", @@ -78,9 +78,9 @@ go_library( "//pkg/ddl/syncer", "//pkg/ddl/util", "//pkg/distsql", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/handle", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/taskexecutor", "//pkg/disttask/framework/taskexecutor/execute", @@ -183,7 +183,7 @@ go_test( timeout = "moderate", srcs = [ "attributes_sql_test.go", - "backfilling_dispatcher_test.go", + "backfilling_dist_scheduler_test.go", "backfilling_test.go", "bench_test.go", "cancel_test.go", @@ -253,8 +253,8 @@ go_test( "//pkg/ddl/testutil", "//pkg/ddl/util", "//pkg/ddl/util/callback", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/domain", "//pkg/domain/infosync", diff --git a/pkg/ddl/backfilling_clean_s3.go b/pkg/ddl/backfilling_clean_s3.go index 39d6d2bf2ebde..c7a73efeafd29 100644 --- a/pkg/ddl/backfilling_clean_s3.go +++ b/pkg/ddl/backfilling_clean_s3.go @@ -21,20 +21,20 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/backend/external" "github.com/pingcap/tidb/br/pkg/storage" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/parser/ast" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" ) -var _ dispatcher.CleanUpRoutine = (*BackfillCleanUpS3)(nil) +var _ scheduler.CleanUpRoutine = (*BackfillCleanUpS3)(nil) -// BackfillCleanUpS3 implements dispatcher.CleanUpRoutine. +// BackfillCleanUpS3 implements scheduler.CleanUpRoutine. type BackfillCleanUpS3 struct { } -func newBackfillCleanUpS3() dispatcher.CleanUpRoutine { +func newBackfillCleanUpS3() scheduler.CleanUpRoutine { return &BackfillCleanUpS3{} } diff --git a/pkg/ddl/backfilling_dispatcher.go b/pkg/ddl/backfilling_dist_scheduler.go similarity index 84% rename from pkg/ddl/backfilling_dispatcher.go rename to pkg/ddl/backfilling_dist_scheduler.go index 092d305158653..06eac3461f443 100644 --- a/pkg/ddl/backfilling_dispatcher.go +++ b/pkg/ddl/backfilling_dist_scheduler.go @@ -28,8 +28,8 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/ddl/ingest" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/meta" @@ -42,33 +42,33 @@ import ( "go.uber.org/zap" ) -// BackfillingDispatcherExt is an extension of litBackfillDispatcher, exported for test. -type BackfillingDispatcherExt struct { +// BackfillingSchedulerExt is an extension of litBackfillScheduler, exported for test. +type BackfillingSchedulerExt struct { d *ddl GlobalSort bool } -// NewBackfillingDispatcherExt creates a new backfillingDispatcherExt, only used for test now. -func NewBackfillingDispatcherExt(d DDL) (dispatcher.Extension, error) { +// NewBackfillingSchedulerExt creates a new backfillingSchedulerExt, only used for test now. +func NewBackfillingSchedulerExt(d DDL) (scheduler.Extension, error) { ddl, ok := d.(*ddl) if !ok { return nil, errors.New("The getDDL result should be the type of *ddl") } - return &BackfillingDispatcherExt{ + return &BackfillingSchedulerExt{ d: ddl, }, nil } -var _ dispatcher.Extension = (*BackfillingDispatcherExt)(nil) +var _ scheduler.Extension = (*BackfillingSchedulerExt)(nil) -// OnTick implements dispatcher.Extension interface. -func (*BackfillingDispatcherExt) OnTick(_ context.Context, _ *proto.Task) { +// OnTick implements scheduler.Extension interface. +func (*BackfillingSchedulerExt) OnTick(_ context.Context, _ *proto.Task) { } // OnNextSubtasksBatch generate batch of next step's plan. -func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( +func (sch *BackfillingSchedulerExt) OnNextSubtasksBatch( ctx context.Context, - taskHandle dispatcher.TaskHandle, + taskHandle scheduler.TaskHandle, task *proto.Task, serverInfo []*infosync.ServerInfo, nextStep proto.Step, @@ -84,7 +84,7 @@ func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( return nil, err } job := &backfillMeta.Job - tblInfo, err := getTblInfo(dsp.d, job) + tblInfo, err := getTblInfo(sch.d, job) if err != nil { return nil, err } @@ -96,7 +96,7 @@ func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( if tblInfo.Partition != nil { return generatePartitionPlan(tblInfo) } - return generateNonPartitionPlan(dsp.d, tblInfo, job, dsp.GlobalSort, len(serverInfo)) + return generateNonPartitionPlan(sch.d, tblInfo, job, sch.GlobalSort, len(serverInfo)) case StepMergeSort: res, err := generateMergePlan(taskHandle, task, logger) if err != nil { @@ -110,7 +110,7 @@ func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( } return res, nil case StepWriteAndIngest: - if dsp.GlobalSort { + if sch.GlobalSort { prevStep := StepReadIndex if backfillMeta.UseMergeSort { prevStep = StepMergeSort @@ -149,13 +149,13 @@ func updateMeta(task *proto.Task, taskMeta *BackfillTaskMeta) error { return nil } -// GetNextStep implements dispatcher.Extension interface. -func (dsp *BackfillingDispatcherExt) GetNextStep(task *proto.Task) proto.Step { +// GetNextStep implements scheduler.Extension interface. +func (sch *BackfillingSchedulerExt) GetNextStep(task *proto.Task) proto.Step { switch task.Step { case proto.StepInit: return StepReadIndex case StepReadIndex: - if dsp.GlobalSort { + if sch.GlobalSort { return StepMergeSort } return proto.StepDone @@ -175,55 +175,55 @@ func skipMergeSort(stats []external.MultipleFilesStat) bool { return external.GetMaxOverlappingTotal(stats) <= external.MergeSortOverlapThreshold } -// OnDone implements dispatcher.Extension interface. -func (*BackfillingDispatcherExt) OnDone(_ context.Context, _ dispatcher.TaskHandle, _ *proto.Task) error { +// OnDone implements scheduler.Extension interface. +func (*BackfillingSchedulerExt) OnDone(_ context.Context, _ scheduler.TaskHandle, _ *proto.Task) error { return nil } -// GetEligibleInstances implements dispatcher.Extension interface. -func (*BackfillingDispatcherExt) GetEligibleInstances(ctx context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - serverInfos, err := dispatcher.GenerateTaskExecutorNodes(ctx) +// GetEligibleInstances implements scheduler.Extension interface. +func (*BackfillingSchedulerExt) GetEligibleInstances(ctx context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + serverInfos, err := scheduler.GenerateTaskExecutorNodes(ctx) if err != nil { return nil, true, err } return serverInfos, true, nil } -// IsRetryableErr implements dispatcher.Extension.IsRetryableErr interface. -func (*BackfillingDispatcherExt) IsRetryableErr(error) bool { +// IsRetryableErr implements scheduler.Extension.IsRetryableErr interface. +func (*BackfillingSchedulerExt) IsRetryableErr(error) bool { return true } -// LitBackfillDispatcher wraps BaseDispatcher. -type LitBackfillDispatcher struct { - *dispatcher.BaseDispatcher +// LitBackfillScheduler wraps BaseScheduler. +type LitBackfillScheduler struct { + *scheduler.BaseScheduler d *ddl } -func newLitBackfillDispatcher(ctx context.Context, d *ddl, taskMgr dispatcher.TaskManager, - serverID string, task *proto.Task) dispatcher.Dispatcher { - dsp := LitBackfillDispatcher{ - d: d, - BaseDispatcher: dispatcher.NewBaseDispatcher(ctx, taskMgr, serverID, task), +func newLitBackfillScheduler(ctx context.Context, d *ddl, taskMgr scheduler.TaskManager, + serverID string, task *proto.Task) scheduler.Scheduler { + sch := LitBackfillScheduler{ + d: d, + BaseScheduler: scheduler.NewBaseScheduler(ctx, taskMgr, serverID, task), } - return &dsp + return &sch } -// Init implements BaseDispatcher interface. -func (dsp *LitBackfillDispatcher) Init() (err error) { +// Init implements BaseScheduler interface. +func (sch *LitBackfillScheduler) Init() (err error) { taskMeta := &BackfillTaskMeta{} - if err = json.Unmarshal(dsp.BaseDispatcher.Task.Meta, taskMeta); err != nil { + if err = json.Unmarshal(sch.BaseScheduler.Task.Meta, taskMeta); err != nil { return errors.Annotate(err, "unmarshal task meta failed") } - dsp.BaseDispatcher.Extension = &BackfillingDispatcherExt{ - d: dsp.d, + sch.BaseScheduler.Extension = &BackfillingSchedulerExt{ + d: sch.d, GlobalSort: len(taskMeta.CloudStorageURI) > 0} - return dsp.BaseDispatcher.Init() + return sch.BaseScheduler.Init() } -// Close implements BaseDispatcher interface. -func (dsp *LitBackfillDispatcher) Close() { - dsp.BaseDispatcher.Close() +// Close implements BaseScheduler interface. +func (sch *LitBackfillScheduler) Close() { + sch.BaseScheduler.Close() } func getTblInfo(d *ddl, job *model.Job) (tblInfo *model.TableInfo, err error) { @@ -338,7 +338,7 @@ func calculateRegionBatch(totalRegionCnt int, instanceCnt int, useLocalDisk bool func generateGlobalSortIngestPlan( ctx context.Context, - taskHandle dispatcher.TaskHandle, + taskHandle scheduler.TaskHandle, task *proto.Task, jobID int64, cloudStorageURI string, @@ -349,7 +349,7 @@ func generateGlobalSortIngestPlan( if err != nil { return nil, err } - instanceIDs, err := dispatcher.GenerateTaskExecutorNodes(ctx) + instanceIDs, err := scheduler.GenerateTaskExecutorNodes(ctx) if err != nil { return nil, err } @@ -409,7 +409,7 @@ func generateGlobalSortIngestPlan( } func generateMergePlan( - taskHandle dispatcher.TaskHandle, + taskHandle scheduler.TaskHandle, task *proto.Task, logger *zap.Logger, ) ([][]byte, error) { @@ -502,7 +502,7 @@ func getRangeSplitter( } func getSummaryFromLastStep( - taskHandle dispatcher.TaskHandle, + taskHandle scheduler.TaskHandle, gTaskID int64, step proto.Step, ) (startKey, endKey kv.Key, totalKVSize uint64, dataFiles, statFiles []string, err error) { diff --git a/pkg/ddl/backfilling_dispatcher_test.go b/pkg/ddl/backfilling_dist_scheduler_test.go similarity index 87% rename from pkg/ddl/backfilling_dispatcher_test.go rename to pkg/ddl/backfilling_dist_scheduler_test.go index 70e6c734648ac..9ffef47149cb6 100644 --- a/pkg/ddl/backfilling_dispatcher_test.go +++ b/pkg/ddl/backfilling_dist_scheduler_test.go @@ -25,8 +25,8 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/backend/external" "github.com/pingcap/tidb/pkg/ddl" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/domain" "github.com/pingcap/tidb/pkg/meta" @@ -38,7 +38,7 @@ import ( "github.com/tikv/client-go/v2/util" ) -func TestBackfillingDispatcherLocalMode(t *testing.T) { +func TestBackfillingSchedulerLocalMode(t *testing.T) { /// test str require.Equal(t, "init", ddl.StepStr(proto.StepInit)) require.Equal(t, "read-index", ddl.StepStr(ddl.StepReadIndex)) @@ -48,7 +48,7 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { require.Equal(t, "unknown", ddl.StepStr(111)) store, dom := testkit.CreateMockStoreAndDomain(t) - dsp, err := ddl.NewBackfillingDispatcherExt(dom.DDL()) + sch, err := ddl.NewBackfillingSchedulerExt(dom.DDL()) require.NoError(t, err) tk := testkit.NewTestKit(t, store) tk.MustExec("use test") @@ -67,11 +67,11 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { tblInfo := tbl.Meta() // 1.1 OnNextSubtasksBatch - task.Step = dsp.GetNextStep(task) + task.Step = sch.GetNextStep(task) require.Equal(t, ddl.StepReadIndex, task.Step) - serverInfos, _, err := dsp.GetEligibleInstances(context.Background(), task) + serverInfos, _, err := sch.GetEligibleInstances(context.Background(), task) require.NoError(t, err) - metas, err := dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) + metas, err := sch.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, len(tblInfo.Partition.Definitions), len(metas)) for i, par := range tblInfo.Partition.Definitions { @@ -82,21 +82,21 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { // 1.2 test partition table OnNextSubtasksBatch after StepReadIndex task.State = proto.TaskStateRunning - task.Step = dsp.GetNextStep(task) + task.Step = sch.GetNextStep(task) require.Equal(t, proto.StepDone, task.Step) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) + metas, err = sch.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Len(t, metas, 0) // 1.3 test partition table OnDone. - err = dsp.OnDone(context.Background(), nil, task) + err = sch.OnDone(context.Background(), nil, task) require.NoError(t, err) /// 2. test non partition table. // 2.1 empty table tk.MustExec("create table t1(id int primary key, v int)") task = createAddIndexTask(t, dom, "test", "t1", proto.Backfill, false) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) + metas, err = sch.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, 0, len(metas)) // 2.2 non empty table. @@ -107,16 +107,16 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { tk.MustExec("insert into t2 values (), (), (), (), (), ()") task = createAddIndexTask(t, dom, "test", "t2", proto.Backfill, false) // 2.2.1 stepInit - task.Step = dsp.GetNextStep(task) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) + task.Step = sch.GetNextStep(task) + metas, err = sch.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, 1, len(metas)) require.Equal(t, ddl.StepReadIndex, task.Step) // 2.2.2 StepReadIndex task.State = proto.TaskStateRunning - task.Step = dsp.GetNextStep(task) + task.Step = sch.GetNextStep(task) require.Equal(t, proto.StepDone, task.Step) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) + metas, err = sch.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, 0, len(metas)) } @@ -141,7 +141,7 @@ func TestCalculateRegionBatch(t *testing.T) { require.Equal(t, 2, batchCnt) } -func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { +func TestBackfillingSchedulerGlobalSortMode(t *testing.T) { // init test env. store, dom := testkit.CreateMockStoreAndDomain(t) tk := testkit.NewTestKit(t, store) @@ -153,7 +153,7 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { ctx = util.WithInternalSourceType(ctx, "handle") mgr := storage.NewTaskManager(pool) storage.SetTaskManager(mgr) - dspManager, err := dispatcher.NewManager(util.WithInternalSourceType(ctx, "dispatcher"), mgr, "host:port") + schManager, err := scheduler.NewManager(util.WithInternalSourceType(ctx, "scheduler"), mgr, "host:port") require.NoError(t, err) tk.MustExec("use test") @@ -164,20 +164,20 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { tk.MustExec("insert into t1 values (), (), (), (), (), ()") task := createAddIndexTask(t, dom, "test", "t1", proto.Backfill, true) - dsp := dspManager.MockDispatcher(task) - ext, err := ddl.NewBackfillingDispatcherExt(dom.DDL()) + sch := schManager.MockScheduler(task) + ext, err := ddl.NewBackfillingSchedulerExt(dom.DDL()) require.NoError(t, err) - ext.(*ddl.BackfillingDispatcherExt).GlobalSort = true - dsp.Extension = ext + ext.(*ddl.BackfillingSchedulerExt).GlobalSort = true + sch.Extension = ext taskID, err := mgr.CreateTask(ctx, task.Key, proto.Backfill, 1, task.Meta) require.NoError(t, err) task.ID = taskID - serverInfos, _, err := dsp.GetEligibleInstances(context.Background(), task) + serverInfos, _, err := sch.GetEligibleInstances(context.Background(), task) require.NoError(t, err) // 1. to read-index stage - subtaskMetas, err := dsp.OnNextSubtasksBatch(ctx, dsp, task, serverInfos, dsp.GetNextStep(task)) + subtaskMetas, err := sch.OnNextSubtasksBatch(ctx, sch, task, serverInfos, sch.GetNextStep(task)) require.NoError(t, err) require.Len(t, subtaskMetas, 1) task.Step = ext.GetNextStep(task) @@ -217,7 +217,7 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { t.Cleanup(func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/forceMergeSort")) }) - subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, dsp, task, serverInfos, ext.GetNextStep(task)) + subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, sch, task, serverInfos, ext.GetNextStep(task)) require.NoError(t, err) require.Len(t, subtaskMetas, 1) task.Step = ext.GetNextStep(task) @@ -256,13 +256,13 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { t.Cleanup(func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockWriteIngest")) }) - subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, dsp, task, serverInfos, ext.GetNextStep(task)) + subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, sch, task, serverInfos, ext.GetNextStep(task)) require.NoError(t, err) require.Len(t, subtaskMetas, 1) task.Step = ext.GetNextStep(task) require.Equal(t, ddl.StepWriteAndIngest, task.Step) // 4. to done stage. - subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, dsp, task, serverInfos, ext.GetNextStep(task)) + subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, sch, task, serverInfos, ext.GetNextStep(task)) require.NoError(t, err) require.Len(t, subtaskMetas, 0) task.Step = ext.GetNextStep(task) @@ -273,7 +273,7 @@ func TestGetNextStep(t *testing.T) { task := &proto.Task{ Step: proto.StepInit, } - ext := &ddl.BackfillingDispatcherExt{} + ext := &ddl.BackfillingSchedulerExt{} // 1. local mode for _, nextStep := range []proto.Step{ddl.StepReadIndex, proto.StepDone} { @@ -281,7 +281,7 @@ func TestGetNextStep(t *testing.T) { task.Step = nextStep } // 2. global sort mode - ext = &ddl.BackfillingDispatcherExt{GlobalSort: true} + ext = &ddl.BackfillingSchedulerExt{GlobalSort: true} task.Step = proto.StepInit for _, nextStep := range []proto.Step{ddl.StepReadIndex, ddl.StepMergeSort, ddl.StepWriteAndIngest} { require.Equal(t, nextStep, ext.GetNextStep(task)) diff --git a/pkg/ddl/ddl.go b/pkg/ddl/ddl.go index 84d393178af40..556dc5641405b 100644 --- a/pkg/ddl/ddl.go +++ b/pkg/ddl/ddl.go @@ -40,8 +40,8 @@ import ( sess "github.com/pingcap/tidb/pkg/ddl/internal/session" "github.com/pingcap/tidb/pkg/ddl/syncer" "github.com/pingcap/tidb/pkg/ddl/util" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/infoschema" @@ -683,11 +683,11 @@ func newDDL(ctx context.Context, options ...Option) *ddl { }, taskexecutor.WithSummary, ) - dispatcher.RegisterDispatcherFactory(proto.Backfill, - func(ctx context.Context, taskMgr dispatcher.TaskManager, serverID string, task *proto.Task) dispatcher.Dispatcher { - return newLitBackfillDispatcher(ctx, d, taskMgr, serverID, task) + scheduler.RegisterSchedulerFactory(proto.Backfill, + func(ctx context.Context, taskMgr scheduler.TaskManager, serverID string, task *proto.Task) scheduler.Scheduler { + return newLitBackfillScheduler(ctx, d, taskMgr, serverID, task) }) - dispatcher.RegisterDispatcherCleanUpFactory(proto.Backfill, newBackfillCleanUpS3) + scheduler.RegisterSchedulerCleanUpFactory(proto.Backfill, newBackfillCleanUpS3) // Register functions for enable/disable ddl when changing system variable `tidb_enable_ddl`. variable.EnableDDL = d.EnableDDL variable.DisableDDL = d.DisableDDL diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 2568b9e38f0a5..eddabb4cfd30d 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -36,9 +36,9 @@ import ( "github.com/pingcap/tidb/pkg/ddl/copr" "github.com/pingcap/tidb/pkg/ddl/ingest" sess "github.com/pingcap/tidb/pkg/ddl/internal/session" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/handle" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/infoschema" "github.com/pingcap/tidb/pkg/kv" @@ -2090,8 +2090,8 @@ func (w *worker) executeDistTask(reorgInfo *reorgInfo) error { } g.Go(func() error { defer close(done) - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - err := handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logutil.BgLogger(), + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + err := handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logutil.BgLogger(), func(ctx context.Context) (bool, error) { return true, handle.ResumeTask(w.ctx, taskKey) }, diff --git a/pkg/ddl/ingest/tests/partition_table_test.go b/pkg/ddl/ingest/tests/partition_table_test.go index 10d7ffbc43ebe..585e08e09a1db 100644 --- a/pkg/ddl/ingest/tests/partition_table_test.go +++ b/pkg/ddl/ingest/tests/partition_table_test.go @@ -42,7 +42,7 @@ func TestAddIndexIngestRecoverPartition(t *testing.T) { tc.SetOwner(1) // TODO(tangenta): mock multiple backends in a better way. //nolint: forcetypeassert - // TODO(tangenta): When owner changes, wait last ddl owner's DDL dispatching loop exits. + // TODO(tangenta): When owner changes, wait last ddl owner's DDL scheduling loop exits. ingest.LitBackCtxMgr.(*ingest.MockBackendCtxMgr).ResetSessCtx() bc, _ := ingest.LitBackCtxMgr.Load(job.ID) bc.GetCheckpointManager().Close() diff --git a/pkg/disttask/framework/BUILD.bazel b/pkg/disttask/framework/BUILD.bazel index a0ea67cd870d2..976f43838219b 100644 --- a/pkg/disttask/framework/BUILD.bazel +++ b/pkg/disttask/framework/BUILD.bazel @@ -15,9 +15,9 @@ go_test( race = "off", shard_count = 32, deps = [ - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/handle", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/testutil", "//pkg/testkit", diff --git a/pkg/disttask/framework/framework_dynamic_dispatch_test.go b/pkg/disttask/framework/framework_dynamic_dispatch_test.go index 30d26418eff0a..228920215869e 100644 --- a/pkg/disttask/framework/framework_dynamic_dispatch_test.go +++ b/pkg/disttask/framework/framework_dynamic_dispatch_test.go @@ -36,16 +36,16 @@ func TestFrameworkDynamicHA(t *testing.T) { defer ctrl.Finish() testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockDynamicDispatchExt(ctrl), testContext, nil) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDynamicDispatchErr", "5*return()")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDynamicDispatchErr", "5*return()")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDynamicDispatchErr")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDynamicDispatchErr")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDynamicDispatchErr1", "5*return()")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDynamicDispatchErr1", "5*return()")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key2", testContext, nil) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDynamicDispatchErr1")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDynamicDispatchErr1")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDynamicDispatchErr2", "5*return()")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDynamicDispatchErr2", "5*return()")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key3", testContext, nil) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDynamicDispatchErr2")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDynamicDispatchErr2")) distContext.Close() } diff --git a/pkg/disttask/framework/framework_err_handling_test.go b/pkg/disttask/framework/framework_err_handling_test.go index 8b5c5e88cd108..57435d643fc8a 100644 --- a/pkg/disttask/framework/framework_err_handling_test.go +++ b/pkg/disttask/framework/framework_err_handling_test.go @@ -24,7 +24,7 @@ import ( func TestPlanErr(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetPlanErrDispatcherExt(ctrl, testContext), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetPlanErrSchedulerExt(ctrl, testContext), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) testContext.CallTime = 0 distContext.Close() @@ -33,7 +33,7 @@ func TestPlanErr(t *testing.T) { func TestRevertPlanErr(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetPlanErrDispatcherExt(ctrl, testContext), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetPlanErrSchedulerExt(ctrl, testContext), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) testContext.CallTime = 0 distContext.Close() @@ -43,7 +43,7 @@ func TestPlanNotRetryableErr(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetPlanNotRetryableErrDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetPlanNotRetryableErrSchedulerExt(ctrl), testContext, nil) testutil.DispatchTaskAndCheckState(ctx, t, "key1", testContext, proto.TaskStateFailed) distContext.Close() } diff --git a/pkg/disttask/framework/framework_ha_test.go b/pkg/disttask/framework/framework_ha_test.go index 54a2990db5c6a..3dc7415138d25 100644 --- a/pkg/disttask/framework/framework_ha_test.go +++ b/pkg/disttask/framework/framework_ha_test.go @@ -26,7 +26,7 @@ func TestHABasic(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 4) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockCleanExecutor", "return()")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockStopManager", "4*return()")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBDown", "return(\":4000\")")) @@ -41,7 +41,7 @@ func TestHAManyNodes(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 30) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockCleanExecutor", "return()")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockStopManager", "30*return()")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBDown", "return(\":4000\")")) @@ -56,7 +56,7 @@ func TestHAFailInDifferentStage(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 6) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestSchedulerExt(ctrl), testContext, nil) // stage1 : server num from 6 to 3. // stage2 : server num from 3 to 2. require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockCleanExecutor", "return()")) @@ -76,7 +76,7 @@ func TestHAFailInDifferentStageManyNodes(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 30) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestSchedulerExt(ctrl), testContext, nil) // stage1 : server num from 30 to 27. // stage2 : server num from 27 to 26. require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockCleanExecutor", "return()")) @@ -96,7 +96,7 @@ func TestHAReplacedButRunning(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 4) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBPartitionThenResume", "10*return(true)")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "😊", testContext, nil) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBPartitionThenResume")) @@ -107,7 +107,7 @@ func TestHAReplacedButRunningManyNodes(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 30) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockHATestSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBPartitionThenResume", "30*return(true)")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "😊", testContext, nil) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBPartitionThenResume")) diff --git a/pkg/disttask/framework/framework_pause_and_resume_test.go b/pkg/disttask/framework/framework_pause_and_resume_test.go index 952327b2809cd..1b63077eb135a 100644 --- a/pkg/disttask/framework/framework_pause_and_resume_test.go +++ b/pkg/disttask/framework/framework_pause_and_resume_test.go @@ -19,9 +19,9 @@ import ( "testing" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/handle" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/framework/testutil" "github.com/stretchr/testify/require" @@ -42,18 +42,18 @@ func TestFrameworkPauseAndResume(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) - // 1. dispatch and pause one running task. - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/pauseTaskAfterRefreshTask", "2*return(true)")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/syncAfterResume", "return()")) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) + // 1. schedule and pause one running task. + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pauseTaskAfterRefreshTask", "2*return(true)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncAfterResume", "return()")) testutil.DispatchTaskAndCheckState(ctx, t, "key1", testContext, proto.TaskStatePaused) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/pauseTaskAfterRefreshTask")) - // 4 subtask dispatched. + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pauseTaskAfterRefreshTask")) + // 4 subtask scheduled. require.NoError(t, handle.ResumeTask(ctx, "key1")) - <-dispatcher.TestSyncChan + <-scheduler.TestSyncChan testutil.WaitTaskExit(ctx, t, "key1") CheckSubtasksState(ctx, t, 1, proto.TaskStateSucceed, 4) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/syncAfterResume")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncAfterResume")) mgr, err := storage.GetTaskManager() require.NoError(t, err) @@ -62,17 +62,17 @@ func TestFrameworkPauseAndResume(t *testing.T) { require.Empty(t, errs) // 2. pause pending task. - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/pausePendingTask", "2*return(true)")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/syncAfterResume", "1*return()")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pausePendingTask", "2*return(true)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncAfterResume", "1*return()")) testutil.DispatchTaskAndCheckState(ctx, t, "key2", testContext, proto.TaskStatePaused) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/pausePendingTask")) - // 4 subtask dispatched. + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pausePendingTask")) + // 4 subtask scheduled. require.NoError(t, handle.ResumeTask(ctx, "key2")) - <-dispatcher.TestSyncChan + <-scheduler.TestSyncChan testutil.WaitTaskExit(ctx, t, "key2") CheckSubtasksState(ctx, t, 1, proto.TaskStateSucceed, 4) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/syncAfterResume")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncAfterResume")) errs, err = mgr.CollectSubTaskError(ctx, 1) require.NoError(t, err) diff --git a/pkg/disttask/framework/framework_rollback_test.go b/pkg/disttask/framework/framework_rollback_test.go index e1d1b61c66d25..247f504cc15ae 100644 --- a/pkg/disttask/framework/framework_rollback_test.go +++ b/pkg/disttask/framework/framework_rollback_test.go @@ -26,10 +26,10 @@ import ( func TestFrameworkRollback(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterRollbackTaskMeta(t, ctrl, testutil.GetMockRollbackDispatcherExt(ctrl), testContext) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/cancelTaskAfterRefreshTask", "2*return(true)")) + testutil.RegisterRollbackTaskMeta(t, ctrl, testutil.GetMockRollbackSchedulerExt(ctrl), testContext) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelTaskAfterRefreshTask", "2*return(true)")) defer func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/cancelTaskAfterRefreshTask")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelTaskAfterRefreshTask")) }() testutil.DispatchTaskAndCheckState(ctx, t, "key1", testContext, proto.TaskStateReverted) diff --git a/pkg/disttask/framework/framework_test.go b/pkg/disttask/framework/framework_test.go index 0d36ce2464b9f..b2a7fc61590d5 100644 --- a/pkg/disttask/framework/framework_test.go +++ b/pkg/disttask/framework/framework_test.go @@ -19,8 +19,8 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/framework/testutil" "github.com/pingcap/tidb/pkg/testkit" @@ -31,7 +31,7 @@ func TestFrameworkBasic(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key2", testContext, nil) distContext.SetOwner(0) @@ -47,7 +47,7 @@ func TestFrameworkBasic(t *testing.T) { func TestFramework3Server(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key2", testContext, nil) distContext.SetOwner(0) @@ -61,7 +61,7 @@ func TestFrameworkAddDomain(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) distContext.AddDomain() testutil.DispatchTaskAndCheckSuccess(ctx, t, "key2", testContext, nil) @@ -77,7 +77,7 @@ func TestFrameworkDeleteDomain(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) distContext.DeleteDomain(1) time.Sleep(2 * time.Second) // make sure the owner changed @@ -89,7 +89,7 @@ func TestFrameworkWithQuery(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) tk := testkit.NewTestKit(t, distContext.Store) @@ -110,7 +110,7 @@ func TestFrameworkCancelGTask(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 2) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchAndCancelTask(ctx, t, "key1", testContext) distContext.Close() } @@ -119,7 +119,7 @@ func TestFrameworkSubTaskFailed(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 1) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunErr", "1*return(true)")) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunErr")) @@ -132,7 +132,7 @@ func TestFrameworkSubTaskFailed(t *testing.T) { func TestFrameworkSubTaskInitEnvFailed(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 1) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockExecSubtaskInitEnvErr", "return()")) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockExecSubtaskInitEnvErr")) @@ -144,13 +144,13 @@ func TestFrameworkSubTaskInitEnvFailed(t *testing.T) { func TestOwnerChange(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) - dispatcher.MockOwnerChange = func() { + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) + scheduler.MockOwnerChange = func() { distContext.SetOwner(0) } - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockOwnerChange", "1*return(true)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockOwnerChange", "1*return(true)")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "😊", testContext, nil) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockOwnerChange")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockOwnerChange")) distContext.Close() } @@ -158,10 +158,10 @@ func TestFrameworkCancelThenSubmitSubTask(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/cancelBeforeUpdate", "return()")) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelBeforeUpdate", "return()")) testutil.DispatchTaskAndCheckState(ctx, t, "😊", testContext, proto.TaskStateReverted) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/cancelBeforeUpdate")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelBeforeUpdate")) distContext.Close() } @@ -169,7 +169,7 @@ func TestTaskExecutorDownBasic(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 4) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockCleanExecutor", "return()")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockStopManager", "4*return(\":4000\")")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBDown", "return(\":4000\")")) @@ -183,7 +183,7 @@ func TestTaskExecutorDownBasic(t *testing.T) { func TestTaskExecutorDownManyNodes(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 30) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockCleanExecutor", "return()")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockStopManager", "30*return(\":4000\")")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBDown", "return(\":4000\")")) @@ -198,7 +198,7 @@ func TestFrameworkSetLabel(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) tk := testkit.NewTestKit(t, distContext.Store) // 1. all "" role. @@ -229,7 +229,7 @@ func TestMultiTasks(t *testing.T) { defer ctrl.Finish() testContext := &testutil.TestContext{} - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) testutil.DispatchMultiTasksAndOneFail(ctx, t, 3, testContext) distContext.Close() @@ -239,13 +239,13 @@ func TestGC(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/storage/subtaskHistoryKeepSeconds", "return(1)")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/historySubtaskTableGcInterval", "return(1)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/historySubtaskTableGcInterval", "return(1)")) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/storage/subtaskHistoryKeepSeconds")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/historySubtaskTableGcInterval")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/historySubtaskTableGcInterval")) }() testutil.DispatchTaskAndCheckSuccess(ctx, t, "😊", testContext, nil) @@ -262,7 +262,7 @@ func TestGC(t *testing.T) { return historySubTasksCnt == 4 }, 10*time.Second, 500*time.Millisecond) - dispatcher.WaitTaskFinished <- struct{}{} + scheduler.WaitTaskFinished <- struct{}{} require.Eventually(t, func() bool { historySubTasksCnt, err := storage.GetSubtasksFromHistoryForTest(ctx, mgr) @@ -279,7 +279,7 @@ func TestFrameworkSubtaskFinishedCancel(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockSubtaskFinishedCancel", "1*return(true)")) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockSubtaskFinishedCancel")) @@ -292,7 +292,7 @@ func TestFrameworkRunSubtaskCancel(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockRunSubtaskCancel", "1*return(true)")) testutil.DispatchTaskAndCheckState(ctx, t, "key1", testContext, proto.TaskStateReverted) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockRunSubtaskCancel")) @@ -302,10 +302,10 @@ func TestFrameworkRunSubtaskCancel(t *testing.T) { func TestFrameworkCleanUpRoutine(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 3) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/WaitCleanUpFinished", "return()")) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished", "return()")) testutil.DispatchTaskAndCheckSuccess(ctx, t, "key1", testContext, nil) - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished mgr, err := storage.GetTaskManager() require.NoError(t, err) tasks, err := mgr.GetTaskByKeyWithHistory(ctx, "key1") @@ -318,9 +318,9 @@ func TestTaskCancelledBeforeUpdateTask(t *testing.T) { ctx, ctrl, testContext, distContext := testutil.InitTestContext(t, 1) defer ctrl.Finish() - testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicDispatcherExt(ctrl), testContext, nil) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/cancelBeforeUpdateTask", "1*return(true)")) + testutil.RegisterTaskMeta(t, ctrl, testutil.GetMockBasicSchedulerExt(ctrl), testContext, nil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelBeforeUpdateTask", "1*return(true)")) testutil.DispatchTaskAndCheckState(ctx, t, "key1", testContext, proto.TaskStateReverted) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/cancelBeforeUpdateTask")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelBeforeUpdateTask")) distContext.Close() } diff --git a/pkg/disttask/framework/handle/handle_test.go b/pkg/disttask/framework/handle/handle_test.go index 5ff6c6c0ddb30..0e6ee2f8a79e9 100644 --- a/pkg/disttask/framework/handle/handle_test.go +++ b/pkg/disttask/framework/handle/handle_test.go @@ -46,7 +46,7 @@ func TestHandle(t *testing.T) { mgr := storage.NewTaskManager(pool) storage.SetTaskManager(mgr) - // no dispatcher registered + // no scheduler registered err := handle.SubmitAndWaitTask(ctx, "1", proto.TaskTypeExample, 2, []byte("byte")) require.Error(t, err) @@ -55,7 +55,7 @@ func TestHandle(t *testing.T) { require.Equal(t, int64(1), task.ID) require.Equal(t, "1", task.Key) require.Equal(t, proto.TaskTypeExample, task.Type) - // no dispatcher registered + // no scheduler registered require.Equal(t, proto.TaskStateFailed, task.State) require.Equal(t, proto.StepInit, task.Step) require.Equal(t, 2, task.Concurrency) diff --git a/pkg/disttask/framework/mock/BUILD.bazel b/pkg/disttask/framework/mock/BUILD.bazel index 014edad728419..fd3ab164569da 100644 --- a/pkg/disttask/framework/mock/BUILD.bazel +++ b/pkg/disttask/framework/mock/BUILD.bazel @@ -3,8 +3,8 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "mock", srcs = [ - "dispatcher_mock.go", "plan_mock.go", + "scheduler_mock.go", "task_executor_mock.go", ], importpath = "github.com/pingcap/tidb/pkg/disttask/framework/mock", diff --git a/pkg/disttask/framework/mock/dispatcher_mock.go b/pkg/disttask/framework/mock/scheduler_mock.go similarity index 94% rename from pkg/disttask/framework/mock/dispatcher_mock.go rename to pkg/disttask/framework/mock/scheduler_mock.go index 0e2fb12a6be74..44235f62c6081 100644 --- a/pkg/disttask/framework/mock/dispatcher_mock.go +++ b/pkg/disttask/framework/mock/scheduler_mock.go @@ -1,9 +1,9 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/pingcap/tidb/pkg/disttask/framework/dispatcher (interfaces: Dispatcher,CleanUpRoutine,TaskManager) +// Source: github.com/pingcap/tidb/pkg/disttask/framework/scheduler (interfaces: Scheduler,CleanUpRoutine,TaskManager) // // Generated by this command: // -// mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/dispatcher Dispatcher,CleanUpRoutine,TaskManager +// mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/scheduler Scheduler,CleanUpRoutine,TaskManager // // Package mock is a generated GoMock package. package mock @@ -17,55 +17,55 @@ import ( gomock "go.uber.org/mock/gomock" ) -// MockDispatcher is a mock of Dispatcher interface. -type MockDispatcher struct { +// MockScheduler is a mock of Scheduler interface. +type MockScheduler struct { ctrl *gomock.Controller - recorder *MockDispatcherMockRecorder + recorder *MockSchedulerMockRecorder } -// MockDispatcherMockRecorder is the mock recorder for MockDispatcher. -type MockDispatcherMockRecorder struct { - mock *MockDispatcher +// MockSchedulerMockRecorder is the mock recorder for MockScheduler. +type MockSchedulerMockRecorder struct { + mock *MockScheduler } -// NewMockDispatcher creates a new mock instance. -func NewMockDispatcher(ctrl *gomock.Controller) *MockDispatcher { - mock := &MockDispatcher{ctrl: ctrl} - mock.recorder = &MockDispatcherMockRecorder{mock} +// NewMockScheduler creates a new mock instance. +func NewMockScheduler(ctrl *gomock.Controller) *MockScheduler { + mock := &MockScheduler{ctrl: ctrl} + mock.recorder = &MockSchedulerMockRecorder{mock} return mock } // EXPECT returns an object that allows the caller to indicate expected use. -func (m *MockDispatcher) EXPECT() *MockDispatcherMockRecorder { +func (m *MockScheduler) EXPECT() *MockSchedulerMockRecorder { return m.recorder } // Close mocks base method. -func (m *MockDispatcher) Close() { +func (m *MockScheduler) Close() { m.ctrl.T.Helper() m.ctrl.Call(m, "Close") } // Close indicates an expected call of Close. -func (mr *MockDispatcherMockRecorder) Close() *gomock.Call { +func (mr *MockSchedulerMockRecorder) Close() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockDispatcher)(nil).Close)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Close", reflect.TypeOf((*MockScheduler)(nil).Close)) } // ExecuteTask mocks base method. -func (m *MockDispatcher) ExecuteTask() { +func (m *MockScheduler) ExecuteTask() { m.ctrl.T.Helper() m.ctrl.Call(m, "ExecuteTask") } // ExecuteTask indicates an expected call of ExecuteTask. -func (mr *MockDispatcherMockRecorder) ExecuteTask() *gomock.Call { +func (mr *MockSchedulerMockRecorder) ExecuteTask() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExecuteTask", reflect.TypeOf((*MockDispatcher)(nil).ExecuteTask)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "ExecuteTask", reflect.TypeOf((*MockScheduler)(nil).ExecuteTask)) } // Init mocks base method. -func (m *MockDispatcher) Init() error { +func (m *MockScheduler) Init() error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "Init") ret0, _ := ret[0].(error) @@ -73,9 +73,9 @@ func (m *MockDispatcher) Init() error { } // Init indicates an expected call of Init. -func (mr *MockDispatcherMockRecorder) Init() *gomock.Call { +func (mr *MockSchedulerMockRecorder) Init() *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Init", reflect.TypeOf((*MockDispatcher)(nil).Init)) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Init", reflect.TypeOf((*MockScheduler)(nil).Init)) } // MockCleanUpRoutine is a mock of CleanUpRoutine interface. diff --git a/pkg/disttask/framework/proto/task.go b/pkg/disttask/framework/proto/task.go index 22ee3991bb403..b97ad7d728cc0 100644 --- a/pkg/disttask/framework/proto/task.go +++ b/pkg/disttask/framework/proto/task.go @@ -147,8 +147,8 @@ type Task struct { // depends on query, below fields might not be filled. - // DispatcherID is not used now. - DispatcherID string + // SchedulerID is not used now. + SchedulerID string StartTime time.Time StateUpdateTime time.Time Meta []byte @@ -176,7 +176,7 @@ func (t *Task) Compare(other *Task) int { } // Subtask represents the subtask of distribute framework. -// Each task is divided into multiple subtasks by dispatcher. +// Each task is divided into multiple subtasks by scheduler. type Subtask struct { ID int64 Step Step diff --git a/pkg/disttask/framework/dispatcher/BUILD.bazel b/pkg/disttask/framework/scheduler/BUILD.bazel similarity index 86% rename from pkg/disttask/framework/dispatcher/BUILD.bazel rename to pkg/disttask/framework/scheduler/BUILD.bazel index e4992f060b0a4..f139fb5415a57 100644 --- a/pkg/disttask/framework/dispatcher/BUILD.bazel +++ b/pkg/disttask/framework/scheduler/BUILD.bazel @@ -1,15 +1,15 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") go_library( - name = "dispatcher", + name = "scheduler", srcs = [ - "dispatcher.go", - "dispatcher_manager.go", "interface.go", + "scheduler.go", + "scheduler_manager.go", "slots.go", "state_transform.go", ], - importpath = "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher", + importpath = "github.com/pingcap/tidb/pkg/disttask/framework/scheduler", visibility = ["//visibility:public"], deps = [ "//pkg/disttask/framework/proto", @@ -32,23 +32,23 @@ go_library( ) go_test( - name = "dispatcher_test", + name = "scheduler_test", timeout = "short", srcs = [ - "dispatcher_manager_test.go", - "dispatcher_test.go", "main_test.go", "rebalance_test.go", + "scheduler_manager_test.go", + "scheduler_test.go", "slots_test.go", ], - embed = [":dispatcher"], + embed = [":scheduler"], flaky = True, race = "off", shard_count = 22, deps = [ - "//pkg/disttask/framework/dispatcher/mock", "//pkg/disttask/framework/mock", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler/mock", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/testutil", "//pkg/domain/infosync", diff --git a/pkg/disttask/framework/dispatcher/interface.go b/pkg/disttask/framework/scheduler/interface.go similarity index 77% rename from pkg/disttask/framework/dispatcher/interface.go rename to pkg/disttask/framework/scheduler/interface.go index 3e42dd6faac4a..61e1bbd76f2cf 100644 --- a/pkg/disttask/framework/dispatcher/interface.go +++ b/pkg/disttask/framework/scheduler/interface.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "context" @@ -67,8 +67,8 @@ type TaskManager interface { } // Extension is used to control the process operations for each task. -// it's used to extend functions of BaseDispatcher. -// as golang doesn't support inheritance, we embed this interface in Dispatcher +// it's used to extend functions of BaseScheduler. +// as golang doesn't support inheritance, we embed this interface in Scheduler // to simulate abstract method as in other OO languages. type Extension interface { // OnTick is used to handle the ticker event, if business impl need to do some periodical work, you can @@ -80,13 +80,13 @@ type Extension interface { // NOTE: don't change task.State inside, framework will manage it. // it's called when: // 1. task is pending and entering it's first step. - // 2. subtasks dispatched has all finished with no error. + // 2. subtasks scheduled has all finished with no error. // when next step is StepDone, it should return nil, nil. OnNextSubtasksBatch(ctx context.Context, h TaskHandle, task *proto.Task, serverInfo []*infosync.ServerInfo, step proto.Step) (subtaskMetas [][]byte, err error) // OnDone is called when task is done, either finished successfully or failed // with error. - // if the task is failed when initializing dispatcher, or it's an unknown task, + // if the task is failed when initializing scheduler, or it's an unknown task, // we don't call this function. OnDone(ctx context.Context, h TaskHandle, task *proto.Task) error @@ -95,49 +95,49 @@ type Extension interface { // The bool return value indicates whether filter instances by role. GetEligibleInstances(ctx context.Context, task *proto.Task) ([]*infosync.ServerInfo, bool, error) - // IsRetryableErr is used to check whether the error occurred in dispatcher is retryable. + // IsRetryableErr is used to check whether the error occurred in scheduler is retryable. IsRetryableErr(err error) bool // GetNextStep is used to get the next step for the task. // if task runs successfully, it should go from StepInit to business steps, - // then to StepDone, then dispatcher will mark it as finished. + // then to StepDone, then scheduler will mark it as finished. GetNextStep(task *proto.Task) proto.Step } -// dispatcherFactoryFn is used to create a dispatcher. -type dispatcherFactoryFn func(ctx context.Context, taskMgr TaskManager, serverID string, task *proto.Task) Dispatcher +// schedulerFactoryFn is used to create a scheduler. +type schedulerFactoryFn func(ctx context.Context, taskMgr TaskManager, serverID string, task *proto.Task) Scheduler -var dispatcherFactoryMap = struct { +var schedulerFactoryMap = struct { syncutil.RWMutex - m map[proto.TaskType]dispatcherFactoryFn + m map[proto.TaskType]schedulerFactoryFn }{ - m: make(map[proto.TaskType]dispatcherFactoryFn), + m: make(map[proto.TaskType]schedulerFactoryFn), } -// RegisterDispatcherFactory is used to register the dispatcher factory. -// normally dispatcher ctor should be registered before the server start. +// RegisterSchedulerFactory is used to register the scheduler factory. +// normally scheduler ctor should be registered before the server start. // and should be called in a single routine, such as in init(). // after the server start, there's should be no write to the map. // but for index backfill, the register call stack is so deep, not sure // if it's safe to do so, so we use a lock here. -func RegisterDispatcherFactory(taskType proto.TaskType, ctor dispatcherFactoryFn) { - dispatcherFactoryMap.Lock() - defer dispatcherFactoryMap.Unlock() - dispatcherFactoryMap.m[taskType] = ctor +func RegisterSchedulerFactory(taskType proto.TaskType, ctor schedulerFactoryFn) { + schedulerFactoryMap.Lock() + defer schedulerFactoryMap.Unlock() + schedulerFactoryMap.m[taskType] = ctor } -// getDispatcherFactory is used to get the dispatcher factory. -func getDispatcherFactory(taskType proto.TaskType) dispatcherFactoryFn { - dispatcherFactoryMap.RLock() - defer dispatcherFactoryMap.RUnlock() - return dispatcherFactoryMap.m[taskType] +// getSchedulerFactory is used to get the scheduler factory. +func getSchedulerFactory(taskType proto.TaskType) schedulerFactoryFn { + schedulerFactoryMap.RLock() + defer schedulerFactoryMap.RUnlock() + return schedulerFactoryMap.m[taskType] } -// ClearDispatcherFactory is only used in test. -func ClearDispatcherFactory() { - dispatcherFactoryMap.Lock() - defer dispatcherFactoryMap.Unlock() - dispatcherFactoryMap.m = make(map[proto.TaskType]dispatcherFactoryFn) +// ClearSchedulerFactory is only used in test. +func ClearSchedulerFactory() { + schedulerFactoryMap.Lock() + defer schedulerFactoryMap.Unlock() + schedulerFactoryMap.m = make(map[proto.TaskType]schedulerFactoryFn) } // CleanUpRoutine is used for the framework to do some clean up work if the task is finished. @@ -155,24 +155,24 @@ var cleanUpFactoryMap = struct { m: make(map[proto.TaskType]cleanUpFactoryFn), } -// RegisterDispatcherCleanUpFactory is used to register the dispatcher clean up factory. -// normally dispatcher cleanup is used in the dispatcher_manager gcTaskLoop to do clean up +// RegisterSchedulerCleanUpFactory is used to register the scheduler clean up factory. +// normally scheduler cleanup is used in the scheduler_manager gcTaskLoop to do clean up // works when tasks are finished. -func RegisterDispatcherCleanUpFactory(taskType proto.TaskType, ctor cleanUpFactoryFn) { +func RegisterSchedulerCleanUpFactory(taskType proto.TaskType, ctor cleanUpFactoryFn) { cleanUpFactoryMap.Lock() defer cleanUpFactoryMap.Unlock() cleanUpFactoryMap.m[taskType] = ctor } -// getDispatcherCleanUpFactory is used to get the dispatcher factory. -func getDispatcherCleanUpFactory(taskType proto.TaskType) cleanUpFactoryFn { +// getSchedulerCleanUpFactory is used to get the scheduler factory. +func getSchedulerCleanUpFactory(taskType proto.TaskType) cleanUpFactoryFn { cleanUpFactoryMap.RLock() defer cleanUpFactoryMap.RUnlock() return cleanUpFactoryMap.m[taskType] } -// ClearDispatcherCleanUpFactory is only used in test. -func ClearDispatcherCleanUpFactory() { +// ClearSchedulerCleanUpFactory is only used in test. +func ClearSchedulerCleanUpFactory() { cleanUpFactoryMap.Lock() defer cleanUpFactoryMap.Unlock() cleanUpFactoryMap.m = make(map[proto.TaskType]cleanUpFactoryFn) diff --git a/pkg/disttask/framework/dispatcher/main_test.go b/pkg/disttask/framework/scheduler/main_test.go similarity index 81% rename from pkg/disttask/framework/dispatcher/main_test.go rename to pkg/disttask/framework/scheduler/main_test.go index 7f452b8b8a319..280c98034910a 100644 --- a/pkg/disttask/framework/dispatcher/main_test.go +++ b/pkg/disttask/framework/scheduler/main_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "testing" @@ -21,24 +21,24 @@ import ( "go.uber.org/goleak" ) -// DispatcherForTest exports for testing. -type DispatcherManagerForTest interface { +// SchedulerForTest exports for testing. +type SchedulerManagerForTest interface { GetRunningTaskCnt() int DelRunningTask(id int64) DoCleanUpRoutine() } -// GetRunningGTaskCnt implements Dispatcher.GetRunningGTaskCnt interface. +// GetRunningGTaskCnt implements Scheduler.GetRunningGTaskCnt interface. func (dm *Manager) GetRunningTaskCnt() int { - return dm.getDispatcherCount() + return dm.getSchedulerCount() } -// DelRunningGTask implements Dispatcher.DelRunningGTask interface. +// DelRunningGTask implements Scheduler.DelRunningGTask interface. func (dm *Manager) DelRunningTask(id int64) { - dm.delDispatcher(id) + dm.delScheduler(id) } -// DoCleanUpRoutine implements Dispatcher.DoCleanUpRoutine interface. +// DoCleanUpRoutine implements Scheduler.DoCleanUpRoutine interface. func (dm *Manager) DoCleanUpRoutine() { dm.doCleanUpRoutine() } diff --git a/pkg/disttask/framework/dispatcher/mock/BUILD.bazel b/pkg/disttask/framework/scheduler/mock/BUILD.bazel similarity index 76% rename from pkg/disttask/framework/dispatcher/mock/BUILD.bazel rename to pkg/disttask/framework/scheduler/mock/BUILD.bazel index d3f1a3dd79c7e..890488b52257b 100644 --- a/pkg/disttask/framework/dispatcher/mock/BUILD.bazel +++ b/pkg/disttask/framework/scheduler/mock/BUILD.bazel @@ -2,12 +2,12 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library") go_library( name = "mock", - srcs = ["dispatcher_mock.go"], - importpath = "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mock", + srcs = ["scheduler_mock.go"], + importpath = "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mock", visibility = ["//visibility:public"], deps = [ - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/domain/infosync", "@org_uber_go_mock//gomock", ], diff --git a/pkg/disttask/framework/dispatcher/mock/dispatcher_mock.go b/pkg/disttask/framework/scheduler/mock/scheduler_mock.go similarity index 91% rename from pkg/disttask/framework/dispatcher/mock/dispatcher_mock.go rename to pkg/disttask/framework/scheduler/mock/scheduler_mock.go index e965b70d4a5a4..f7ea143c00c78 100644 --- a/pkg/disttask/framework/dispatcher/mock/dispatcher_mock.go +++ b/pkg/disttask/framework/scheduler/mock/scheduler_mock.go @@ -1,9 +1,9 @@ // Code generated by MockGen. DO NOT EDIT. -// Source: github.com/pingcap/tidb/pkg/disttask/framework/dispatcher (interfaces: Extension) +// Source: github.com/pingcap/tidb/pkg/disttask/framework/scheduler (interfaces: Extension) // // Generated by this command: // -// mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/dispatcher Extension +// mockgen -package mock github.com/pingcap/tidb/pkg/disttask/framework/scheduler Extension // // Package mock is a generated GoMock package. package mock @@ -12,8 +12,8 @@ import ( context "context" reflect "reflect" - dispatcher "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" proto "github.com/pingcap/tidb/pkg/disttask/framework/proto" + scheduler "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" infosync "github.com/pingcap/tidb/pkg/domain/infosync" gomock "go.uber.org/mock/gomock" ) @@ -86,7 +86,7 @@ func (mr *MockExtensionMockRecorder) IsRetryableErr(arg0 any) *gomock.Call { } // OnDone mocks base method. -func (m *MockExtension) OnDone(arg0 context.Context, arg1 dispatcher.TaskHandle, arg2 *proto.Task) error { +func (m *MockExtension) OnDone(arg0 context.Context, arg1 scheduler.TaskHandle, arg2 *proto.Task) error { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "OnDone", arg0, arg1, arg2) ret0, _ := ret[0].(error) @@ -100,7 +100,7 @@ func (mr *MockExtensionMockRecorder) OnDone(arg0, arg1, arg2 any) *gomock.Call { } // OnNextSubtasksBatch mocks base method. -func (m *MockExtension) OnNextSubtasksBatch(arg0 context.Context, arg1 dispatcher.TaskHandle, arg2 *proto.Task, arg3 []*infosync.ServerInfo, arg4 proto.Step) ([][]byte, error) { +func (m *MockExtension) OnNextSubtasksBatch(arg0 context.Context, arg1 scheduler.TaskHandle, arg2 *proto.Task, arg3 []*infosync.ServerInfo, arg4 proto.Step) ([][]byte, error) { m.ctrl.T.Helper() ret := m.ctrl.Call(m, "OnNextSubtasksBatch", arg0, arg1, arg2, arg3, arg4) ret0, _ := ret[0].([][]byte) diff --git a/pkg/disttask/framework/dispatcher/rebalance_test.go b/pkg/disttask/framework/scheduler/rebalance_test.go similarity index 96% rename from pkg/disttask/framework/dispatcher/rebalance_test.go rename to pkg/disttask/framework/scheduler/rebalance_test.go index 530cc5a56ee07..f11efc8393c21 100644 --- a/pkg/disttask/framework/dispatcher/rebalance_test.go +++ b/pkg/disttask/framework/scheduler/rebalance_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher_test +package scheduler_test import ( "context" @@ -22,9 +22,9 @@ import ( "time" "github.com/ngaut/pools" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/mock" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/testkit" "github.com/stretchr/testify/require" @@ -60,17 +60,17 @@ func scaleTest(t *testing.T, if len(testCase.cleanedNodes) > 0 { mockTaskMgr.EXPECT().GetSubtasksByExecIdsAndStepAndState(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil, nil) } - dsp := dispatcher.NewBaseDispatcher(ctx, mockTaskMgr, "server", &proto.Task{Step: proto.StepInit, ID: int64(id)}) - dsp.LiveNodes = testCase.liveNodes - dsp.TaskNodes = testCase.taskNodes - require.NoError(t, dsp.ReDispatchSubtasks()) - slices.SortFunc(dsp.TaskNodes, func(i, j string) int { + sch := scheduler.NewBaseScheduler(ctx, mockTaskMgr, "server", &proto.Task{Step: proto.StepInit, ID: int64(id)}) + sch.LiveNodes = testCase.liveNodes + sch.TaskNodes = testCase.taskNodes + require.NoError(t, sch.ReDispatchSubtasks()) + slices.SortFunc(sch.TaskNodes, func(i, j string) int { return strings.Compare(i, j) }) slices.SortFunc(testCase.subtasks, func(i, j *proto.Subtask) int { return strings.Compare(i.ExecID, j.ExecID) }) - require.Equal(t, testCase.expectedTaskNodes, dsp.TaskNodes) + require.Equal(t, testCase.expectedTaskNodes, sch.TaskNodes) require.Equal(t, testCase.expectedSubtasks, testCase.subtasks) } @@ -85,11 +85,11 @@ func balanceTest(t *testing.T, mockTaskMgr.EXPECT().CleanUpMeta(ctx, gomock.Any()).Return(nil).AnyTimes() mockTaskMgr.EXPECT().UpdateSubtasksExecIDs(ctx, int64(id), testCase.subtasks).Return(nil).AnyTimes() - dsp := dispatcher.NewBaseDispatcher(ctx, mockTaskMgr, "server", &proto.Task{Step: proto.StepInit, ID: int64(id)}) - dsp.LiveNodes = testCase.liveNodes - dsp.TaskNodes = testCase.taskNodes - require.NoError(t, dsp.ReDispatchSubtasks()) - slices.SortFunc(dsp.TaskNodes, func(i, j string) int { + sch := scheduler.NewBaseScheduler(ctx, mockTaskMgr, "server", &proto.Task{Step: proto.StepInit, ID: int64(id)}) + sch.LiveNodes = testCase.liveNodes + sch.TaskNodes = testCase.taskNodes + require.NoError(t, sch.ReDispatchSubtasks()) + slices.SortFunc(sch.TaskNodes, func(i, j string) int { return strings.Compare(i, j) }) slices.SortFunc(testCase.subtasks, func(i, j *proto.Subtask) int { diff --git a/pkg/disttask/framework/dispatcher/dispatcher.go b/pkg/disttask/framework/scheduler/scheduler.go similarity index 55% rename from pkg/disttask/framework/dispatcher/dispatcher.go rename to pkg/disttask/framework/scheduler/scheduler.go index c0c0a77b7d703..e9a43edf22474 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher.go +++ b/pkg/disttask/framework/scheduler/scheduler.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "context" @@ -38,7 +38,7 @@ const ( DefaultSubtaskConcurrency = 16 // MaxSubtaskConcurrency is the maximum concurrency for handling subtask. MaxSubtaskConcurrency = 256 - // DefaultLiveNodesCheckInterval is the tick interval of fetching all server infos from etcd. + // DefaultLiveNodesCheckInterval is the tick interval of fetching all server infos from etcs. DefaultLiveNodesCheckInterval = 2 // for a cancelled task, it's terminal state is reverted or reverted_failed, // so we use a special error message to indicate that the task is cancelled @@ -57,8 +57,8 @@ var ( RetrySQLMaxInterval = 30 * time.Second ) -// TaskHandle provides the interface for operations needed by Dispatcher. -// Then we can use dispatcher's function in Dispatcher interface. +// TaskHandle provides the interface for operations needed by Scheduler. +// Then we can use scheduler's function in Scheduler interface. type TaskHandle interface { // GetPreviousTaskExecutorIDs gets previous task executor IDs. GetPreviousTaskExecutorIDs(_ context.Context, taskID int64, step proto.Step) ([]string, error) @@ -67,29 +67,29 @@ type TaskHandle interface { storage.SessionExecutor } -// Dispatcher manages the lifetime of a task +// Scheduler manages the lifetime of a task // including submitting subtasks and updating the status of a task. -type Dispatcher interface { - // Init initializes the dispatcher, should be called before ExecuteTask. - // if Init returns error, dispatcher manager will fail the task directly, +type Scheduler interface { + // Init initializes the scheduler, should be called before ExecuteTask. + // if Init returns error, scheduler manager will fail the task directly, // so the returned error should be a fatal error. Init() error // ExecuteTask start to schedule a task. ExecuteTask() - // Close closes the dispatcher, should be called if Init returns nil. + // Close closes the scheduler, should be called if Init returns nil. Close() } -// BaseDispatcher is the base struct for Dispatcher. +// BaseScheduler is the base struct for Scheduler. // each task type embed this struct and implement the Extension interface. -type BaseDispatcher struct { +type BaseScheduler struct { ctx context.Context taskMgr TaskManager Task *proto.Task logCtx context.Context // serverID, it's value is ip:port now. serverID string - // when RegisterDispatcherFactory, the factory MUST initialize this field. + // when RegisterSchedulerFactory, the factory MUST initialize this fields. Extension // For subtasks rebalance. @@ -107,11 +107,11 @@ type BaseDispatcher struct { // MockOwnerChange mock owner change in tests. var MockOwnerChange func() -// NewBaseDispatcher creates a new BaseDispatcher. -func NewBaseDispatcher(ctx context.Context, taskMgr TaskManager, serverID string, task *proto.Task) *BaseDispatcher { +// NewBaseScheduler creates a new BaseScheduler. +func NewBaseScheduler(ctx context.Context, taskMgr TaskManager, serverID string, task *proto.Task) *BaseScheduler { logCtx := logutil.WithFields(context.Background(), zap.Int64("task-id", task.ID), zap.Stringer("task-type", task.Type)) - return &BaseDispatcher{ + return &BaseScheduler{ ctx: ctx, taskMgr: taskMgr, Task: task, @@ -125,111 +125,111 @@ func NewBaseDispatcher(ctx context.Context, taskMgr TaskManager, serverID string } } -// Init implements the Dispatcher interface. -func (*BaseDispatcher) Init() error { +// Init implements the Scheduler interface. +func (*BaseScheduler) Init() error { return nil } -// ExecuteTask implements the Dispatcher interface. -func (d *BaseDispatcher) ExecuteTask() { - logutil.Logger(d.logCtx).Info("execute one task", - zap.Stringer("state", d.Task.State), zap.Int("concurrency", d.Task.Concurrency)) - d.scheduleTask() +// ExecuteTask implements the Scheduler interface. +func (s *BaseScheduler) ExecuteTask() { + logutil.Logger(s.logCtx).Info("execute one task", + zap.Stringer("state", s.Task.State), zap.Int("concurrency", s.Task.Concurrency)) + s.scheduleTask() } -// Close closes the dispatcher. -func (*BaseDispatcher) Close() { +// Close closes the scheduler. +func (*BaseScheduler) Close() { } // refreshTask fetch task state from tidb_global_task table. -func (d *BaseDispatcher) refreshTask() error { - newTask, err := d.taskMgr.GetTaskByID(d.ctx, d.Task.ID) +func (s *BaseScheduler) refreshTask() error { + newTask, err := s.taskMgr.GetTaskByID(s.ctx, s.Task.ID) if err != nil { - logutil.Logger(d.logCtx).Error("refresh task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Error("refresh task failed", zap.Error(err)) return err } // newTask might be nil when GC routine move the task into history table. if newTask != nil { - d.Task = newTask + s.Task = newTask } return nil } // scheduleTask schedule the task execution step by step. -func (d *BaseDispatcher) scheduleTask() { +func (s *BaseScheduler) scheduleTask() { ticker := time.NewTicker(checkTaskFinishedInterval) defer ticker.Stop() for { select { - case <-d.ctx.Done(): - logutil.Logger(d.logCtx).Info("schedule task exits", zap.Error(d.ctx.Err())) + case <-s.ctx.Done(): + logutil.Logger(s.logCtx).Info("schedule task exits", zap.Error(s.ctx.Err())) return case <-ticker.C: - err := d.refreshTask() + err := s.refreshTask() if err != nil { continue } failpoint.Inject("cancelTaskAfterRefreshTask", func(val failpoint.Value) { - if val.(bool) && d.Task.State == proto.TaskStateRunning { - err := d.taskMgr.CancelTask(d.ctx, d.Task.ID) + if val.(bool) && s.Task.State == proto.TaskStateRunning { + err := s.taskMgr.CancelTask(s.ctx, s.Task.ID) if err != nil { - logutil.Logger(d.logCtx).Error("cancel task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Error("cancel task failed", zap.Error(err)) } } }) failpoint.Inject("pausePendingTask", func(val failpoint.Value) { - if val.(bool) && d.Task.State == proto.TaskStatePending { - _, err := d.taskMgr.PauseTask(d.ctx, d.Task.Key) + if val.(bool) && s.Task.State == proto.TaskStatePending { + _, err := s.taskMgr.PauseTask(s.ctx, s.Task.Key) if err != nil { - logutil.Logger(d.logCtx).Error("pause task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Error("pause task failed", zap.Error(err)) } - d.Task.State = proto.TaskStatePausing + s.Task.State = proto.TaskStatePausing } }) failpoint.Inject("pauseTaskAfterRefreshTask", func(val failpoint.Value) { - if val.(bool) && d.Task.State == proto.TaskStateRunning { - _, err := d.taskMgr.PauseTask(d.ctx, d.Task.Key) + if val.(bool) && s.Task.State == proto.TaskStateRunning { + _, err := s.taskMgr.PauseTask(s.ctx, s.Task.Key) if err != nil { - logutil.Logger(d.logCtx).Error("pause task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Error("pause task failed", zap.Error(err)) } - d.Task.State = proto.TaskStatePausing + s.Task.State = proto.TaskStatePausing } }) - switch d.Task.State { + switch s.Task.State { case proto.TaskStateCancelling: - err = d.onCancelling() + err = s.onCancelling() case proto.TaskStatePausing: - err = d.onPausing() + err = s.onPausing() case proto.TaskStatePaused: - err = d.onPaused() - // close the dispatcher. + err = s.onPaused() + // close the scheduler. if err == nil { return } case proto.TaskStateResuming: - err = d.onResuming() + err = s.onResuming() case proto.TaskStateReverting: - err = d.onReverting() + err = s.onReverting() case proto.TaskStatePending: - err = d.onPending() + err = s.onPending() case proto.TaskStateRunning: - err = d.onRunning() + err = s.onRunning() case proto.TaskStateSucceed, proto.TaskStateReverted, proto.TaskStateFailed: - if err := d.onFinished(); err != nil { - logutil.Logger(d.logCtx).Error("schedule task meet error", zap.Stringer("state", d.Task.State), zap.Error(err)) + if err := s.onFinished(); err != nil { + logutil.Logger(s.logCtx).Error("schedule task meet error", zap.Stringer("state", s.Task.State), zap.Error(err)) } return } if err != nil { - logutil.Logger(d.logCtx).Info("schedule task meet err, reschedule it", zap.Error(err)) + logutil.Logger(s.logCtx).Info("schedule task meet err, reschedule it", zap.Error(err)) } failpoint.Inject("mockOwnerChange", func(val failpoint.Value) { if val.(bool) { - logutil.Logger(d.logCtx).Info("mockOwnerChange called") + logutil.Logger(s.logCtx).Info("mockOwnerChange called") MockOwnerChange() time.Sleep(time.Second) } @@ -238,38 +238,38 @@ func (d *BaseDispatcher) scheduleTask() { } } -// handle task in cancelling state, dispatch revert subtasks. -func (d *BaseDispatcher) onCancelling() error { - logutil.Logger(d.logCtx).Info("on cancelling state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) +// handle task in cancelling state, schedule revert subtasks. +func (s *BaseScheduler) onCancelling() error { + logutil.Logger(s.logCtx).Info("on cancelling state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) errs := []error{errors.New(taskCancelMsg)} - return d.onErrHandlingStage(errs) + return s.onErrHandlingStage(errs) } // handle task in pausing state, cancel all running subtasks. -func (d *BaseDispatcher) onPausing() error { - logutil.Logger(d.logCtx).Info("on pausing state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) - cnt, err := d.taskMgr.GetSubtaskInStatesCnt(d.ctx, d.Task.ID, proto.TaskStateRunning, proto.TaskStatePending) +func (s *BaseScheduler) onPausing() error { + logutil.Logger(s.logCtx).Info("on pausing state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) + cnt, err := s.taskMgr.GetSubtaskInStatesCnt(s.ctx, s.Task.ID, proto.TaskStateRunning, proto.TaskStatePending) if err != nil { - logutil.Logger(d.logCtx).Warn("check task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("check task failed", zap.Error(err)) return err } if cnt == 0 { - logutil.Logger(d.logCtx).Info("all running subtasks paused, update the task to paused state") - return d.updateTask(proto.TaskStatePaused, nil, RetrySQLTimes) + logutil.Logger(s.logCtx).Info("all running subtasks paused, update the task to paused state") + return s.updateTask(proto.TaskStatePaused, nil, RetrySQLTimes) } - logutil.Logger(d.logCtx).Debug("on pausing state, this task keeps current state", zap.Stringer("state", d.Task.State)) + logutil.Logger(s.logCtx).Debug("on pausing state, this task keeps current state", zap.Stringer("state", s.Task.State)) return nil } -// MockDMLExecutionOnPausedState is used to mock DML execution when tasks paused. +// MockDMLExecutionOnPausedState is used to mock DML execution when tasks pauses. var MockDMLExecutionOnPausedState func(task *proto.Task) // handle task in paused state. -func (d *BaseDispatcher) onPaused() error { - logutil.Logger(d.logCtx).Info("on paused state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) +func (s *BaseScheduler) onPaused() error { + logutil.Logger(s.logCtx).Info("on paused state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) failpoint.Inject("mockDMLExecutionOnPausedState", func(val failpoint.Value) { if val.(bool) { - MockDMLExecutionOnPausedState(d.Task) + MockDMLExecutionOnPausedState(s.Task) } }) return nil @@ -279,116 +279,116 @@ func (d *BaseDispatcher) onPaused() error { var TestSyncChan = make(chan struct{}) // handle task in resuming state. -func (d *BaseDispatcher) onResuming() error { - logutil.Logger(d.logCtx).Info("on resuming state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) - cnt, err := d.taskMgr.GetSubtaskInStatesCnt(d.ctx, d.Task.ID, proto.TaskStatePaused) +func (s *BaseScheduler) onResuming() error { + logutil.Logger(s.logCtx).Info("on resuming state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) + cnt, err := s.taskMgr.GetSubtaskInStatesCnt(s.ctx, s.Task.ID, proto.TaskStatePaused) if err != nil { - logutil.Logger(d.logCtx).Warn("check task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("check task failed", zap.Error(err)) return err } if cnt == 0 { // Finish the resuming process. - logutil.Logger(d.logCtx).Info("all paused tasks converted to pending state, update the task to running state") - err := d.updateTask(proto.TaskStateRunning, nil, RetrySQLTimes) + logutil.Logger(s.logCtx).Info("all paused tasks converted to pending state, update the task to running state") + err := s.updateTask(proto.TaskStateRunning, nil, RetrySQLTimes) failpoint.Inject("syncAfterResume", func() { TestSyncChan <- struct{}{} }) return err } - return d.taskMgr.ResumeSubtasks(d.ctx, d.Task.ID) + return s.taskMgr.ResumeSubtasks(s.ctx, s.Task.ID) } -// handle task in reverting state, check all revert subtasks finished. -func (d *BaseDispatcher) onReverting() error { - logutil.Logger(d.logCtx).Debug("on reverting state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) - cnt, err := d.taskMgr.GetSubtaskInStatesCnt(d.ctx, d.Task.ID, proto.TaskStateRevertPending, proto.TaskStateReverting) +// handle task in reverting state, check all revert subtasks finishes. +func (s *BaseScheduler) onReverting() error { + logutil.Logger(s.logCtx).Debug("on reverting state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) + cnt, err := s.taskMgr.GetSubtaskInStatesCnt(s.ctx, s.Task.ID, proto.TaskStateRevertPending, proto.TaskStateReverting) if err != nil { - logutil.Logger(d.logCtx).Warn("check task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("check task failed", zap.Error(err)) return err } if cnt == 0 { - if err = d.OnDone(d.ctx, d, d.Task); err != nil { + if err = s.OnDone(s.ctx, s, s.Task); err != nil { return errors.Trace(err) } - return d.updateTask(proto.TaskStateReverted, nil, RetrySQLTimes) + return s.updateTask(proto.TaskStateReverted, nil, RetrySQLTimes) } - // Wait all subtasks in this stage finished. - d.OnTick(d.ctx, d.Task) - logutil.Logger(d.logCtx).Debug("on reverting state, this task keeps current state", zap.Stringer("state", d.Task.State)) + // Wait all subtasks in this stage finishes. + s.OnTick(s.ctx, s.Task) + logutil.Logger(s.logCtx).Debug("on reverting state, this task keeps current state", zap.Stringer("state", s.Task.State)) return nil } -// handle task in pending state, dispatch subtasks. -func (d *BaseDispatcher) onPending() error { - logutil.Logger(d.logCtx).Debug("on pending state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) - return d.onNextStage() +// handle task in pending state, schedule subtasks. +func (s *BaseScheduler) onPending() error { + logutil.Logger(s.logCtx).Debug("on pending state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) + return s.onNextStage() } -// handle task in running state, check all running subtasks finished. +// handle task in running state, check all running subtasks finishes. // If subtasks finished, run into the next stage. -func (d *BaseDispatcher) onRunning() error { - logutil.Logger(d.logCtx).Debug("on running state", zap.Stringer("state", d.Task.State), zap.Int64("stage", int64(d.Task.Step))) - subTaskErrs, err := d.taskMgr.CollectSubTaskError(d.ctx, d.Task.ID) +func (s *BaseScheduler) onRunning() error { + logutil.Logger(s.logCtx).Debug("on running state", zap.Stringer("state", s.Task.State), zap.Int64("stage", int64(s.Task.Step))) + subTaskErrs, err := s.taskMgr.CollectSubTaskError(s.ctx, s.Task.ID) if err != nil { - logutil.Logger(d.logCtx).Warn("collect subtask error failed", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("collect subtask error failed", zap.Error(err)) return err } if len(subTaskErrs) > 0 { - logutil.Logger(d.logCtx).Warn("subtasks encounter errors") - return d.onErrHandlingStage(subTaskErrs) + logutil.Logger(s.logCtx).Warn("subtasks encounter errors") + return s.onErrHandlingStage(subTaskErrs) } - // check current stage finished. - cnt, err := d.taskMgr.GetSubtaskInStatesCnt(d.ctx, d.Task.ID, proto.TaskStatePending, proto.TaskStateRunning) + // check current stage finishes. + cnt, err := s.taskMgr.GetSubtaskInStatesCnt(s.ctx, s.Task.ID, proto.TaskStatePending, proto.TaskStateRunning) if err != nil { - logutil.Logger(d.logCtx).Warn("check task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("check task failed", zap.Error(err)) return err } if cnt == 0 { - return d.onNextStage() + return s.onNextStage() } - if err := d.BalanceSubtasks(); err != nil { + if err := s.BalanceSubtasks(); err != nil { return err } - // Wait all subtasks in this stage finished. - d.OnTick(d.ctx, d.Task) - logutil.Logger(d.logCtx).Debug("on running state, this task keeps current state", zap.Stringer("state", d.Task.State)) + // Wait all subtasks in this stage finishes. + s.OnTick(s.ctx, s.Task) + logutil.Logger(s.logCtx).Debug("on running state, this task keeps current state", zap.Stringer("state", s.Task.State)) return nil } -func (d *BaseDispatcher) onFinished() error { - metrics.UpdateMetricsForFinishTask(d.Task) - logutil.Logger(d.logCtx).Debug("schedule task, task is finished", zap.Stringer("state", d.Task.State)) - return d.taskMgr.TransferSubTasks2History(d.ctx, d.Task.ID) +func (s *BaseScheduler) onFinished() error { + metrics.UpdateMetricsForFinishTask(s.Task) + logutil.Logger(s.logCtx).Debug("schedule task, task is finished", zap.Stringer("state", s.Task.State)) + return s.taskMgr.TransferSubTasks2History(s.ctx, s.Task.ID) } // BalanceSubtasks check the liveNode num every liveNodeFetchInterval then rebalance subtasks. -func (d *BaseDispatcher) BalanceSubtasks() error { - // 1. init TaskNodes if needed. - if len(d.TaskNodes) == 0 { +func (s *BaseScheduler) BalanceSubtasks() error { + // 1. init TaskNodes if needes. + if len(s.TaskNodes) == 0 { var err error - d.TaskNodes, err = d.taskMgr.GetTaskExecutorIDsByTaskIDAndStep(d.ctx, d.Task.ID, d.Task.Step) + s.TaskNodes, err = s.taskMgr.GetTaskExecutorIDsByTaskIDAndStep(s.ctx, s.Task.ID, s.Task.Step) if err != nil { return err } } - d.liveNodeFetchTick++ - if d.liveNodeFetchTick == d.liveNodeFetchInterval { + s.liveNodeFetchTick++ + if s.liveNodeFetchTick == s.liveNodeFetchInterval { // 2. update LiveNodes. - d.liveNodeFetchTick = 0 - serverInfos, err := GenerateTaskExecutorNodes(d.ctx) + s.liveNodeFetchTick = 0 + serverInfos, err := GenerateTaskExecutorNodes(s.ctx) if err != nil { return err } - eligibleServerInfos, filter, err := d.GetEligibleInstances(d.ctx, d.Task) + eligibleServerInfos, filter, err := s.GetEligibleInstances(s.ctx, s.Task) if err != nil { return err } if filter { - eligibleServerInfos, err = d.filterByRole(eligibleServerInfos) + eligibleServerInfos, err = s.filterByRole(eligibleServerInfos) if err != nil { return err } @@ -406,52 +406,52 @@ func (d *BaseDispatcher) BalanceSubtasks() error { newInfos = append(newInfos, m) } } - d.LiveNodes = newInfos + s.LiveNodes = newInfos // 3. balance subtasks. - if len(d.LiveNodes) > 0 { - return d.ReDispatchSubtasks() + if len(s.LiveNodes) > 0 { + return s.ReDispatchSubtasks() } return nil } return nil } -func (d *BaseDispatcher) replaceTaskNodes() { - d.TaskNodes = d.TaskNodes[:0] - for _, serverInfo := range d.LiveNodes { - d.TaskNodes = append(d.TaskNodes, disttaskutil.GenerateExecID(serverInfo.IP, serverInfo.Port)) +func (s *BaseScheduler) replaceTaskNodes() { + s.TaskNodes = s.TaskNodes[:0] + for _, serverInfo := range s.LiveNodes { + s.TaskNodes = append(s.TaskNodes, disttaskutil.GenerateExecID(serverInfo.IP, serverInfo.Port)) } } // ReDispatchSubtasks make count of subtasks on each liveNodes balanced and clean up subtasks on dead nodes. // TODO(ywqzzy): refine to make it easier for testing. -func (d *BaseDispatcher) ReDispatchSubtasks() error { +func (s *BaseScheduler) ReDispatchSubtasks() error { // 1. find out nodes need to clean subtasks. deadNodes := make([]string, 0) deadNodesMap := make(map[string]bool, 0) - for _, node := range d.TaskNodes { - if !disttaskutil.MatchServerInfo(d.LiveNodes, node) { + for _, node := range s.TaskNodes { + if !disttaskutil.MatchServerInfo(s.LiveNodes, node) { deadNodes = append(deadNodes, node) deadNodesMap[node] = true } } // 2. get subtasks for each node before rebalance. - subtasks, err := d.taskMgr.GetSubtasksByStepAndState(d.ctx, d.Task.ID, d.Task.Step, proto.TaskStatePending) + subtasks, err := s.taskMgr.GetSubtasksByStepAndState(s.ctx, s.Task.ID, s.Task.Step, proto.TaskStatePending) if err != nil { return err } if len(deadNodes) != 0 { /// get subtask from deadNodes, since there might be some running subtasks on deadNodes. /// In this case, all subtasks on deadNodes are in running/pending state. - subtasksOnDeadNodes, err := d.taskMgr.GetSubtasksByExecIdsAndStepAndState(d.ctx, deadNodes, d.Task.ID, d.Task.Step, proto.TaskStateRunning) + subtasksOnDeadNodes, err := s.taskMgr.GetSubtasksByExecIdsAndStepAndState(s.ctx, deadNodes, s.Task.ID, s.Task.Step, proto.TaskStateRunning) if err != nil { return err } subtasks = append(subtasks, subtasksOnDeadNodes...) } // 3. group subtasks for each task executor. - subtasksOnTaskExecutor := make(map[string][]*proto.Subtask, len(d.LiveNodes)+len(deadNodes)) - for _, node := range d.LiveNodes { + subtasksOnTaskExecutor := make(map[string][]*proto.Subtask, len(s.LiveNodes)+len(deadNodes)) + for _, node := range s.LiveNodes { execID := disttaskutil.GenerateExecID(node.IP, node.Port) subtasksOnTaskExecutor[execID] = make([]*proto.Subtask, 0) } @@ -461,17 +461,17 @@ func (d *BaseDispatcher) ReDispatchSubtasks() error { subtask) } // 4. prepare subtasks that need to rebalance to other nodes. - averageSubtaskCnt := len(subtasks) / len(d.LiveNodes) + averageSubtaskCnt := len(subtasks) / len(s.LiveNodes) rebalanceSubtasks := make([]*proto.Subtask, 0) for k, v := range subtasksOnTaskExecutor { if ok := deadNodesMap[k]; ok { rebalanceSubtasks = append(rebalanceSubtasks, v...) continue } - // When no tidb scale-in/out and averageSubtaskCnt*len(d.LiveNodes) < len(subtasks), + // When no tidb scale-in/out and averageSubtaskCnt*len(s.LiveNodes) < len(subtasks), // no need to send subtask to other nodes. // eg: tidb1 with 3 subtasks, tidb2 with 2 subtasks, subtasks are balanced now. - if averageSubtaskCnt*len(d.LiveNodes) < len(subtasks) && len(d.TaskNodes) == len(d.LiveNodes) { + if averageSubtaskCnt*len(s.LiveNodes) < len(subtasks) && len(s.TaskNodes) == len(s.LiveNodes) { if len(v) > averageSubtaskCnt+1 { rebalanceSubtasks = append(rebalanceSubtasks, v[0:len(v)-averageSubtaskCnt]...) } @@ -500,106 +500,106 @@ func (d *BaseDispatcher) ReDispatchSubtasks() error { // 7. rebalance rest subtasks evenly to liveNodes. liveNodeIdx := 0 for rebalanceIdx < len(rebalanceSubtasks) { - node := d.LiveNodes[liveNodeIdx] + node := s.LiveNodes[liveNodeIdx] rebalanceSubtasks[rebalanceIdx].ExecID = disttaskutil.GenerateExecID(node.IP, node.Port) rebalanceIdx++ liveNodeIdx++ } // 8. update subtasks and do clean up logic. - if err = d.taskMgr.UpdateSubtasksExecIDs(d.ctx, d.Task.ID, subtasks); err != nil { + if err = s.taskMgr.UpdateSubtasksExecIDs(s.ctx, s.Task.ID, subtasks); err != nil { return err } - logutil.Logger(d.logCtx).Info("rebalance subtasks", + logutil.Logger(s.logCtx).Info("rebalance subtasks", zap.Stringers("subtasks-rebalanced", subtasks)) - if err = d.taskMgr.CleanUpMeta(d.ctx, deadNodes); err != nil { + if err = s.taskMgr.CleanUpMeta(s.ctx, deadNodes); err != nil { return err } - d.replaceTaskNodes() + s.replaceTaskNodes() return nil } // updateTask update the task in tidb_global_task table. -func (d *BaseDispatcher) updateTask(taskState proto.TaskState, newSubTasks []*proto.Subtask, retryTimes int) (err error) { - prevState := d.Task.State - d.Task.State = taskState +func (s *BaseScheduler) updateTask(taskState proto.TaskState, newSubTasks []*proto.Subtask, retryTimes int) (err error) { + prevState := s.Task.State + s.Task.State = taskState logutil.BgLogger().Info("task state transform", zap.Stringer("from", prevState), zap.Stringer("to", taskState)) if !VerifyTaskStateTransform(prevState, taskState) { return errors.Errorf("invalid task state transform, from %s to %s", prevState, taskState) } failpoint.Inject("cancelBeforeUpdate", func() { - err := d.taskMgr.CancelTask(d.ctx, d.Task.ID) + err := s.taskMgr.CancelTask(s.ctx, s.Task.ID) if err != nil { - logutil.Logger(d.logCtx).Error("cancel task failed", zap.Error(err)) + logutil.Logger(s.logCtx).Error("cancel task failed", zap.Error(err)) } }) var retryable bool for i := 0; i < retryTimes; i++ { - retryable, err = d.taskMgr.UpdateTaskAndAddSubTasks(d.ctx, d.Task, newSubTasks, prevState) + retryable, err = s.taskMgr.UpdateTaskAndAddSubTasks(s.ctx, s.Task, newSubTasks, prevState) if err == nil || !retryable { break } if i%10 == 0 { - logutil.Logger(d.logCtx).Warn("updateTask first failed", zap.Stringer("from", prevState), zap.Stringer("to", d.Task.State), + logutil.Logger(s.logCtx).Warn("updateTask first failed", zap.Stringer("from", prevState), zap.Stringer("to", s.Task.State), zap.Int("retry times", i), zap.Error(err)) } time.Sleep(RetrySQLInterval) } if err != nil && retryTimes != nonRetrySQLTime { - logutil.Logger(d.logCtx).Warn("updateTask failed", - zap.Stringer("from", prevState), zap.Stringer("to", d.Task.State), zap.Int("retry times", retryTimes), zap.Error(err)) + logutil.Logger(s.logCtx).Warn("updateTask failed", + zap.Stringer("from", prevState), zap.Stringer("to", s.Task.State), zap.Int("retry times", retryTimes), zap.Error(err)) } return err } -func (d *BaseDispatcher) onErrHandlingStage(receiveErrs []error) error { +func (s *BaseScheduler) onErrHandlingStage(receiveErrs []error) error { // we only store the first error. - d.Task.Error = receiveErrs[0] + s.Task.Error = receiveErrs[0] var subTasks []*proto.Subtask // when step of task is `StepInit`, no need to do revert - if d.Task.Step != proto.StepInit { - instanceIDs, err := d.GetAllTaskExecutorIDs(d.ctx, d.Task) + if s.Task.Step != proto.StepInit { + instanceIDs, err := s.GetAllTaskExecutorIDs(s.ctx, s.Task) if err != nil { - logutil.Logger(d.logCtx).Warn("get task's all instances failed", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("get task's all instances failed", zap.Error(err)) return err } subTasks = make([]*proto.Subtask, 0, len(instanceIDs)) for _, id := range instanceIDs { // reverting subtasks belong to the same step as current active step. - subTasks = append(subTasks, proto.NewSubtask(d.Task.Step, d.Task.ID, d.Task.Type, id, d.Task.Concurrency, []byte("{}"))) + subTasks = append(subTasks, proto.NewSubtask(s.Task.Step, s.Task.ID, s.Task.Type, id, s.Task.Concurrency, []byte("{}"))) } } - return d.updateTask(proto.TaskStateReverting, subTasks, RetrySQLTimes) + return s.updateTask(proto.TaskStateReverting, subTasks, RetrySQLTimes) } -func (*BaseDispatcher) nextStepSubtaskDispatched(*proto.Task) bool { - // TODO: will implement it when we we support dispatch subtask by batch. +func (*BaseScheduler) nextStepSubtaskScheduled(*proto.Task) bool { + // TODO: will implement it when we we support schedule subtask by batch. // since subtask meta might be too large to save in one transaction. return true } -func (d *BaseDispatcher) onNextStage() (err error) { - /// dynamic dispatch subtasks. +func (s *BaseScheduler) onNextStage() (err error) { + /// dynamic schedule subtasks. failpoint.Inject("mockDynamicDispatchErr", func() { failpoint.Return(errors.New("mockDynamicDispatchErr")) }) - nextStep := d.GetNextStep(d.Task) - logutil.Logger(d.logCtx).Info("onNextStage", - zap.Int64("current-step", int64(d.Task.Step)), + nextStep := s.GetNextStep(s.Task) + logutil.Logger(s.logCtx).Info("onNextStage", + zap.Int64("current-step", int64(s.Task.Step)), zap.Int64("next-step", int64(nextStep))) // 1. Adjust the task's concurrency. - if d.Task.State == proto.TaskStatePending { - if d.Task.Concurrency == 0 { - d.Task.Concurrency = DefaultSubtaskConcurrency + if s.Task.State == proto.TaskStatePending { + if s.Task.Concurrency == 0 { + s.Task.Concurrency = DefaultSubtaskConcurrency } - if d.Task.Concurrency > MaxSubtaskConcurrency { - d.Task.Concurrency = MaxSubtaskConcurrency + if s.Task.Concurrency > MaxSubtaskConcurrency { + s.Task.Concurrency = MaxSubtaskConcurrency } } defer func() { @@ -612,66 +612,66 @@ func (d *BaseDispatcher) onNextStage() (err error) { // when all subtasks of task.Step is finished, we call OnNextSubtasksBatch // to generate subtasks of next step. after all subtasks of next step are // saved to system table, we will update task.Step to next step, so the - // invariant hold. - // see nextStepSubtaskDispatched for why we don't update task and subtasks + // invariant hols. + // see nextStepSubtaskScheduled for why we don't update task and subtasks // in a single transaction. - if d.nextStepSubtaskDispatched(d.Task) { - currStep := d.Task.Step - d.Task.Step = nextStep - // When all subtasks dispatched and processed, mark task as succeed. + if s.nextStepSubtaskScheduled(s.Task) { + currStep := s.Task.Step + s.Task.Step = nextStep + // When all subtasks scheduled and processed, mark task as succeeds. taskState := proto.TaskStateRunning - if d.Task.Step == proto.StepDone { + if s.Task.Step == proto.StepDone { taskState = proto.TaskStateSucceed - if err = d.OnDone(d.ctx, d, d.Task); err != nil { + if err = s.OnDone(s.ctx, s, s.Task); err != nil { err = errors.Trace(err) return } } else { - logutil.Logger(d.logCtx).Info("move to next stage", - zap.Int64("from", int64(currStep)), zap.Int64("to", int64(d.Task.Step))) + logutil.Logger(s.logCtx).Info("move to next stage", + zap.Int64("from", int64(currStep)), zap.Int64("to", int64(s.Task.Step))) } - d.Task.StateUpdateTime = time.Now().UTC() - err = d.updateTask(taskState, nil, RetrySQLTimes) + s.Task.StateUpdateTime = time.Now().UTC() + err = s.updateTask(taskState, nil, RetrySQLTimes) } }() for { // 3. generate a batch of subtasks. /// select all available TiDB nodes for task. - serverNodes, filter, err := d.GetEligibleInstances(d.ctx, d.Task) - logutil.Logger(d.logCtx).Debug("eligible instances", zap.Int("num", len(serverNodes))) + serverNodes, filter, err := s.GetEligibleInstances(s.ctx, s.Task) + logutil.Logger(s.logCtx).Debug("eligible instances", zap.Int("num", len(serverNodes))) if err != nil { return err } if filter { - serverNodes, err = d.filterByRole(serverNodes) + serverNodes, err = s.filterByRole(serverNodes) if err != nil { return err } } - logutil.Logger(d.logCtx).Info("eligible instances", zap.Int("num", len(serverNodes))) + logutil.Logger(s.logCtx).Info("eligible instances", zap.Int("num", len(serverNodes))) if len(serverNodes) == 0 { - return errors.New("no available TiDB node to dispatch subtasks") + return errors.New("no available TiDB node to schedule subtasks") } - metas, err := d.OnNextSubtasksBatch(d.ctx, d, d.Task, serverNodes, nextStep) + metas, err := s.OnNextSubtasksBatch(s.ctx, s, s.Task, serverNodes, nextStep) if err != nil { - logutil.Logger(d.logCtx).Warn("generate part of subtasks failed", zap.Error(err)) - return d.handlePlanErr(err) + logutil.Logger(s.logCtx).Warn("generate part of subtasks failed", zap.Error(err)) + return s.handlePlanErr(err) } failpoint.Inject("mockDynamicDispatchErr1", func() { failpoint.Return(errors.New("mockDynamicDispatchErr1")) }) - // 4. dispatch batch of subtasks to EligibleInstances. - err = d.dispatchSubTask(nextStep, metas, serverNodes) + // 4. schedule batch of subtasks to EligibleInstances. + err = s.scheduleSubTask(nextStep, metas, serverNodes) if err != nil { return err } - if d.nextStepSubtaskDispatched(d.Task) { + if s.nextStepSubtaskScheduled(s.Task) { break } @@ -682,18 +682,18 @@ func (d *BaseDispatcher) onNextStage() (err error) { return nil } -func (d *BaseDispatcher) dispatchSubTask( +func (s *BaseScheduler) scheduleSubTask( subtaskStep proto.Step, metas [][]byte, serverNodes []*infosync.ServerInfo) error { - logutil.Logger(d.logCtx).Info("dispatch subtasks", - zap.Stringer("state", d.Task.State), - zap.Int64("step", int64(d.Task.Step)), - zap.Int("concurrency", d.Task.Concurrency), + logutil.Logger(s.logCtx).Info("schedule subtasks", + zap.Stringer("state", s.Task.State), + zap.Int64("step", int64(s.Task.Step)), + zap.Int("concurrency", s.Task.Concurrency), zap.Int("subtasks", len(metas))) - d.TaskNodes = make([]string, len(serverNodes)) + s.TaskNodes = make([]string, len(serverNodes)) for i := range serverNodes { - d.TaskNodes[i] = disttaskutil.GenerateExecID(serverNodes[i].IP, serverNodes[i].Port) + s.TaskNodes[i] = disttaskutil.GenerateExecID(serverNodes[i].IP, serverNodes[i].Port) } subTasks := make([]*proto.Subtask, 0, len(metas)) for i, meta := range metas { @@ -701,30 +701,30 @@ func (d *BaseDispatcher) dispatchSubTask( // TODO: assign the subtask to the instance according to the system load of each nodes pos := i % len(serverNodes) instanceID := disttaskutil.GenerateExecID(serverNodes[pos].IP, serverNodes[pos].Port) - logutil.Logger(d.logCtx).Debug("create subtasks", zap.String("instanceID", instanceID)) - subTasks = append(subTasks, proto.NewSubtask(subtaskStep, d.Task.ID, d.Task.Type, instanceID, d.Task.Concurrency, meta)) + logutil.Logger(s.logCtx).Debug("create subtasks", zap.String("instanceID", instanceID)) + subTasks = append(subTasks, proto.NewSubtask(subtaskStep, s.Task.ID, s.Task.Type, instanceID, s.Task.Concurrency, meta)) } failpoint.Inject("cancelBeforeUpdateTask", func() { - _ = d.updateTask(proto.TaskStateCancelling, subTasks, RetrySQLTimes) + _ = s.updateTask(proto.TaskStateCancelling, subTasks, RetrySQLTimes) }) - return d.updateTask(d.Task.State, subTasks, RetrySQLTimes) + return s.updateTask(s.Task.State, subTasks, RetrySQLTimes) } -func (d *BaseDispatcher) handlePlanErr(err error) error { - logutil.Logger(d.logCtx).Warn("generate plan failed", zap.Error(err), zap.Stringer("state", d.Task.State)) - if d.IsRetryableErr(err) { +func (s *BaseScheduler) handlePlanErr(err error) error { + logutil.Logger(s.logCtx).Warn("generate plan failed", zap.Error(err), zap.Stringer("state", s.Task.State)) + if s.IsRetryableErr(err) { return err } - d.Task.Error = err + s.Task.Error = err - if err = d.OnDone(d.ctx, d, d.Task); err != nil { + if err = s.OnDone(s.ctx, s, s.Task); err != nil { return errors.Trace(err) } - return d.updateTask(proto.TaskStateFailed, nil, RetrySQLTimes) + return s.updateTask(proto.TaskStateFailed, nil, RetrySQLTimes) } -// MockServerInfo exported for dispatcher_test.go +// MockServerInfo exported for scheduler_test.go var MockServerInfo []*infosync.ServerInfo // GenerateTaskExecutorNodes generate a eligible TiDB nodes. @@ -753,8 +753,8 @@ func GenerateTaskExecutorNodes(ctx context.Context) (serverNodes []*infosync.Ser return serverNodes, nil } -func (d *BaseDispatcher) filterByRole(infos []*infosync.ServerInfo) ([]*infosync.ServerInfo, error) { - nodes, err := d.taskMgr.GetManagedNodes(d.ctx) +func (s *BaseScheduler) filterByRole(infos []*infosync.ServerInfo) ([]*infosync.ServerInfo, error) { + nodes, err := s.taskMgr.GetManagedNodes(s.ctx) if err != nil { return nil, err } @@ -775,7 +775,7 @@ func (d *BaseDispatcher) filterByRole(infos []*infosync.ServerInfo) ([]*infosync } // GetAllTaskExecutorIDs gets all the task executor IDs. -func (d *BaseDispatcher) GetAllTaskExecutorIDs(ctx context.Context, task *proto.Task) ([]string, error) { +func (s *BaseScheduler) GetAllTaskExecutorIDs(ctx context.Context, task *proto.Task) ([]string, error) { // We get all servers instead of eligible servers here // because eligible servers may change during the task execution. serverInfos, err := GenerateTaskExecutorNodes(ctx) @@ -786,7 +786,7 @@ func (d *BaseDispatcher) GetAllTaskExecutorIDs(ctx context.Context, task *proto. return nil, nil } - executorIDs, err := d.taskMgr.GetTaskExecutorIDsByTaskID(d.ctx, task.ID) + executorIDs, err := s.taskMgr.GetTaskExecutorIDsByTaskID(s.ctx, task.ID) if err != nil { return nil, err } @@ -800,10 +800,10 @@ func (d *BaseDispatcher) GetAllTaskExecutorIDs(ctx context.Context, task *proto. } // GetPreviousSubtaskMetas get subtask metas from specific step. -func (d *BaseDispatcher) GetPreviousSubtaskMetas(taskID int64, step proto.Step) ([][]byte, error) { - previousSubtasks, err := d.taskMgr.GetSubtasksByStepAndState(d.ctx, taskID, step, proto.TaskStateSucceed) +func (s *BaseScheduler) GetPreviousSubtaskMetas(taskID int64, step proto.Step) ([][]byte, error) { + previousSubtasks, err := s.taskMgr.GetSubtasksByStepAndState(s.ctx, taskID, step, proto.TaskStateSucceed) if err != nil { - logutil.Logger(d.logCtx).Warn("get previous succeed subtask failed", zap.Int64("step", int64(step))) + logutil.Logger(s.logCtx).Warn("get previous succeed subtask failed", zap.Int64("step", int64(step))) return nil, err } previousSubtaskMetas := make([][]byte, 0, len(previousSubtasks)) @@ -814,18 +814,18 @@ func (d *BaseDispatcher) GetPreviousSubtaskMetas(taskID int64, step proto.Step) } // GetPreviousTaskExecutorIDs gets task executor IDs that run previous step. -func (d *BaseDispatcher) GetPreviousTaskExecutorIDs(_ context.Context, taskID int64, step proto.Step) ([]string, error) { - return d.taskMgr.GetTaskExecutorIDsByTaskIDAndStep(d.ctx, taskID, step) +func (s *BaseScheduler) GetPreviousTaskExecutorIDs(_ context.Context, taskID int64, step proto.Step) ([]string, error) { + return s.taskMgr.GetTaskExecutorIDsByTaskIDAndStep(s.ctx, taskID, step) } // WithNewSession executes the function with a new session. -func (d *BaseDispatcher) WithNewSession(fn func(se sessionctx.Context) error) error { - return d.taskMgr.WithNewSession(fn) +func (s *BaseScheduler) WithNewSession(fn func(se sessionctx.Context) error) error { + return s.taskMgr.WithNewSession(fn) } // WithNewTxn executes the fn in a new transaction. -func (d *BaseDispatcher) WithNewTxn(ctx context.Context, fn func(se sessionctx.Context) error) error { - return d.taskMgr.WithNewTxn(ctx, fn) +func (s *BaseScheduler) WithNewTxn(ctx context.Context, fn func(se sessionctx.Context) error) error { + return s.taskMgr.WithNewTxn(ctx, fn) } // IsCancelledErr checks if the error is a cancelled error. diff --git a/pkg/disttask/framework/dispatcher/dispatcher_manager.go b/pkg/disttask/framework/scheduler/scheduler_manager.go similarity index 56% rename from pkg/disttask/framework/dispatcher/dispatcher_manager.go rename to pkg/disttask/framework/scheduler/scheduler_manager.go index d0c7e6b6a3214..b5c5ddb709864 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher_manager.go +++ b/pkg/disttask/framework/scheduler/scheduler_manager.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "context" @@ -43,39 +43,39 @@ var ( // WaitTaskFinished is used to sync the test. var WaitTaskFinished = make(chan struct{}) -func (dm *Manager) getDispatcherCount() int { - dm.mu.RLock() - defer dm.mu.RUnlock() - return len(dm.mu.dispatchers) +func (sm *Manager) getSchedulerCount() int { + sm.mu.RLock() + defer sm.mu.RUnlock() + return len(sm.mu.schedulers) } -func (dm *Manager) addDispatcher(taskID int64, dispatcher Dispatcher) { - dm.mu.Lock() - defer dm.mu.Unlock() - dm.mu.dispatchers[taskID] = dispatcher +func (sm *Manager) addScheduler(taskID int64, scheduler Scheduler) { + sm.mu.Lock() + defer sm.mu.Unlock() + sm.mu.schedulers[taskID] = scheduler } -func (dm *Manager) hasDispatcher(taskID int64) bool { - dm.mu.Lock() - defer dm.mu.Unlock() - _, ok := dm.mu.dispatchers[taskID] +func (sm *Manager) hasScheduler(taskID int64) bool { + sm.mu.Lock() + defer sm.mu.Unlock() + _, ok := sm.mu.schedulers[taskID] return ok } -func (dm *Manager) delDispatcher(taskID int64) { - dm.mu.Lock() - defer dm.mu.Unlock() - delete(dm.mu.dispatchers, taskID) +func (sm *Manager) delScheduler(taskID int64) { + sm.mu.Lock() + defer sm.mu.Unlock() + delete(sm.mu.schedulers, taskID) } -func (dm *Manager) clearDispatchers() { - dm.mu.Lock() - defer dm.mu.Unlock() - dm.mu.dispatchers = make(map[int64]Dispatcher) +func (sm *Manager) clearSchedulers() { + sm.mu.Lock() + defer sm.mu.Unlock() + sm.mu.schedulers = make(map[int64]Scheduler) } -// Manager manage a bunch of dispatchers. -// Dispatcher schedule and monitor tasks. +// Manager manage a bunch of schedulers. +// Scheduler schedule and monitor tasks. // The scheduling task number is limited by size of gPool. type Manager struct { ctx context.Context @@ -92,131 +92,131 @@ type Manager struct { mu struct { syncutil.RWMutex - dispatchers map[int64]Dispatcher + schedulers map[int64]Scheduler } } -// NewManager creates a dispatcher struct. +// NewManager creates a scheduler struct. func NewManager(ctx context.Context, taskMgr TaskManager, serverID string) (*Manager, error) { - dispatcherManager := &Manager{ + schedulerManager := &Manager{ taskMgr: taskMgr, serverID: serverID, slotMgr: newSlotManager(), } - gPool, err := spool.NewPool("dispatch_pool", int32(proto.MaxConcurrentTask), util.DistTask, spool.WithBlocking(true)) + gPool, err := spool.NewPool("schedule_pool", int32(proto.MaxConcurrentTask), util.DistTask, spool.WithBlocking(true)) if err != nil { return nil, err } - dispatcherManager.gPool = gPool - dispatcherManager.ctx, dispatcherManager.cancel = context.WithCancel(ctx) - dispatcherManager.mu.dispatchers = make(map[int64]Dispatcher) - dispatcherManager.finishCh = make(chan struct{}, proto.MaxConcurrentTask) + schedulerManager.gPool = gPool + schedulerManager.ctx, schedulerManager.cancel = context.WithCancel(ctx) + schedulerManager.mu.schedulers = make(map[int64]Scheduler) + schedulerManager.finishCh = make(chan struct{}, proto.MaxConcurrentTask) - return dispatcherManager, nil + return schedulerManager, nil } -// Start the dispatcherManager, start the dispatchTaskLoop to start multiple dispatchers. -func (dm *Manager) Start() { - failpoint.Inject("disableDispatcherManager", func() { +// Start the schedulerManager, start the scheduleTaskLoop to start multiple schedulers. +func (sm *Manager) Start() { + failpoint.Inject("disableSchedulerManager", func() { failpoint.Return() }) - dm.wg.Run(dm.dispatchTaskLoop) - dm.wg.Run(dm.gcSubtaskHistoryTableLoop) - dm.wg.Run(dm.cleanUpLoop) - dm.initialized = true + sm.wg.Run(sm.scheduleTaskLoop) + sm.wg.Run(sm.gcSubtaskHistoryTableLoop) + sm.wg.Run(sm.cleanUpLoop) + sm.initialized = true } -// Stop the dispatcherManager. -func (dm *Manager) Stop() { - dm.cancel() - dm.gPool.ReleaseAndWait() - dm.wg.Wait() - dm.clearDispatchers() - dm.initialized = false - close(dm.finishCh) +// Stop the schedulerManager. +func (sm *Manager) Stop() { + sm.cancel() + sm.gPool.ReleaseAndWait() + sm.wg.Wait() + sm.clearSchedulers() + sm.initialized = false + close(sm.finishCh) } // Initialized check the manager initialized. -func (dm *Manager) Initialized() bool { - return dm.initialized +func (sm *Manager) Initialized() bool { + return sm.initialized } -// dispatchTaskLoop dispatches the tasks. -func (dm *Manager) dispatchTaskLoop() { - logutil.BgLogger().Info("dispatch task loop start") +// scheduleTaskLoop schedulees the tasks. +func (sm *Manager) scheduleTaskLoop() { + logutil.BgLogger().Info("schedule task loop start") ticker := time.NewTicker(checkTaskRunningInterval) defer ticker.Stop() for { select { - case <-dm.ctx.Done(): - logutil.BgLogger().Info("dispatch task loop exits", zap.Error(dm.ctx.Err()), zap.Int64("interval", int64(checkTaskRunningInterval)/1000000)) + case <-sm.ctx.Done(): + logutil.BgLogger().Info("schedule task loop exits", zap.Error(sm.ctx.Err()), zap.Int64("interval", int64(checkTaskRunningInterval)/1000000)) return case <-ticker.C: } - taskCnt := dm.getDispatcherCount() + taskCnt := sm.getSchedulerCount() if taskCnt >= proto.MaxConcurrentTask { - logutil.BgLogger().Info("dispatched tasks reached limit", + logutil.BgLogger().Info("scheduled tasks reached limit", zap.Int("current", taskCnt), zap.Int("max", proto.MaxConcurrentTask)) continue } - tasks, err := dm.taskMgr.GetTopUnfinishedTasks(dm.ctx) + tasks, err := sm.taskMgr.GetTopUnfinishedTasks(sm.ctx) if err != nil { logutil.BgLogger().Warn("get unfinished tasks failed", zap.Error(err)) continue } - dispatchableTasks := make([]*proto.Task, 0, len(tasks)) + scheduleableTasks := make([]*proto.Task, 0, len(tasks)) for _, task := range tasks { - if dm.hasDispatcher(task.ID) { + if sm.hasScheduler(task.ID) { continue } - // we check it before start dispatcher, so no need to check it again. - // see startDispatcher. + // we check it before start scheduler, so no need to check it again. + // see startScheduler. // this should not happen normally, unless user modify system table // directly. - if getDispatcherFactory(task.Type) == nil { + if getSchedulerFactory(task.Type) == nil { logutil.BgLogger().Warn("unknown task type", zap.Int64("task-id", task.ID), zap.Stringer("task-type", task.Type)) - dm.failTask(task.ID, task.State, errors.New("unknown task type")) + sm.failTask(task.ID, task.State, errors.New("unknown task type")) continue } - dispatchableTasks = append(dispatchableTasks, task) + scheduleableTasks = append(scheduleableTasks, task) } - if len(dispatchableTasks) == 0 { + if len(scheduleableTasks) == 0 { continue } - if err = dm.slotMgr.update(dm.ctx, dm.taskMgr); err != nil { + if err = sm.slotMgr.update(sm.ctx, sm.taskMgr); err != nil { logutil.BgLogger().Warn("update used slot failed", zap.Error(err)) continue } - for _, task := range dispatchableTasks { - taskCnt = dm.getDispatcherCount() + for _, task := range scheduleableTasks { + taskCnt = sm.getSchedulerCount() if taskCnt >= proto.MaxConcurrentTask { break } - reservedExecID, ok := dm.slotMgr.canReserve(task) + reservedExecID, ok := sm.slotMgr.canReserve(task) if !ok { - // task of lower priority might be able to be dispatched. + // task of lower priority might be able to be scheduled. continue } - metrics.DistTaskGauge.WithLabelValues(task.Type.String(), metrics.DispatchingStatus).Inc() + metrics.DistTaskGauge.WithLabelValues(task.Type.String(), metrics.SchedulingStatus).Inc() metrics.UpdateMetricsForDispatchTask(task.ID, task.Type) - dm.startDispatcher(task, reservedExecID) + sm.startScheduler(task, reservedExecID) } } } -func (dm *Manager) failTask(id int64, currState proto.TaskState, err error) { - if err2 := dm.taskMgr.FailTask(dm.ctx, id, currState, err); err2 != nil { +func (sm *Manager) failTask(id int64, currState proto.TaskState, err error) { + if err2 := sm.taskMgr.FailTask(sm.ctx, id, currState, err); err2 != nil { logutil.BgLogger().Warn("failed to update task state to failed", zap.Int64("task-id", id), zap.Error(err2)) } } -func (dm *Manager) gcSubtaskHistoryTableLoop() { +func (sm *Manager) gcSubtaskHistoryTableLoop() { historySubtaskTableGcInterval := defaultHistorySubtaskTableGcInterval failpoint.Inject("historySubtaskTableGcInterval", func(val failpoint.Value) { if seconds, ok := val.(int); ok { @@ -226,68 +226,68 @@ func (dm *Manager) gcSubtaskHistoryTableLoop() { <-WaitTaskFinished }) - logutil.Logger(dm.ctx).Info("subtask table gc loop start") + logutil.Logger(sm.ctx).Info("subtask table gc loop start") ticker := time.NewTicker(historySubtaskTableGcInterval) defer ticker.Stop() for { select { - case <-dm.ctx.Done(): - logutil.BgLogger().Info("subtask history table gc loop exits", zap.Error(dm.ctx.Err())) + case <-sm.ctx.Done(): + logutil.BgLogger().Info("subtask history table gc loop exits", zap.Error(sm.ctx.Err())) return case <-ticker.C: - err := dm.taskMgr.GCSubtasks(dm.ctx) + err := sm.taskMgr.GCSubtasks(sm.ctx) if err != nil { logutil.BgLogger().Warn("subtask history table gc failed", zap.Error(err)) } else { - logutil.Logger(dm.ctx).Info("subtask history table gc success") + logutil.Logger(sm.ctx).Info("subtask history table gc success") } } } } -func (dm *Manager) startDispatcher(basicTask *proto.Task, reservedExecID string) { - task, err := dm.taskMgr.GetTaskByID(dm.ctx, basicTask.ID) +func (sm *Manager) startScheduler(basicTask *proto.Task, reservedExecID string) { + task, err := sm.taskMgr.GetTaskByID(sm.ctx, basicTask.ID) if err != nil { logutil.BgLogger().Error("get task failed", zap.Error(err)) return } - dispatcherFactory := getDispatcherFactory(task.Type) - dispatcher := dispatcherFactory(dm.ctx, dm.taskMgr, dm.serverID, task) - if err = dispatcher.Init(); err != nil { - logutil.BgLogger().Error("init dispatcher failed", zap.Error(err)) - dm.failTask(task.ID, task.State, err) + schedulerFactory := getSchedulerFactory(task.Type) + scheduler := schedulerFactory(sm.ctx, sm.taskMgr, sm.serverID, task) + if err = scheduler.Init(); err != nil { + logutil.BgLogger().Error("init scheduler failed", zap.Error(err)) + sm.failTask(task.ID, task.State, err) return } - dm.addDispatcher(task.ID, dispatcher) - dm.slotMgr.reserve(basicTask, reservedExecID) + sm.addScheduler(task.ID, scheduler) + sm.slotMgr.reserve(basicTask, reservedExecID) // Using the pool with block, so it wouldn't return an error. - _ = dm.gPool.Run(func() { + _ = sm.gPool.Run(func() { defer func() { - dispatcher.Close() - dm.delDispatcher(task.ID) - dm.slotMgr.unReserve(basicTask, reservedExecID) + scheduler.Close() + sm.delScheduler(task.ID) + sm.slotMgr.unReserve(basicTask, reservedExecID) }() metrics.UpdateMetricsForRunTask(task) - dispatcher.ExecuteTask() + scheduler.ExecuteTask() logutil.BgLogger().Info("task finished", zap.Int64("task-id", task.ID)) - dm.finishCh <- struct{}{} + sm.finishCh <- struct{}{} }) } -func (dm *Manager) cleanUpLoop() { - logutil.Logger(dm.ctx).Info("cleanUp loop start") +func (sm *Manager) cleanUpLoop() { + logutil.Logger(sm.ctx).Info("cleanUp loop start") ticker := time.NewTicker(defaultCleanUpInterval) defer ticker.Stop() for { select { - case <-dm.ctx.Done(): - logutil.BgLogger().Info("cleanUp loop exits", zap.Error(dm.ctx.Err())) + case <-sm.ctx.Done(): + logutil.BgLogger().Info("cleanUp loop exits", zap.Error(sm.ctx.Err())) return - case <-dm.finishCh: - dm.doCleanUpRoutine() + case <-sm.finishCh: + sm.doCleanUpRoutine() case <-ticker.C: - dm.doCleanUpRoutine() + sm.doCleanUpRoutine() } } } @@ -299,13 +299,13 @@ var WaitCleanUpFinished = make(chan struct{}) // For example: // // tasks with global sort should clean up tmp files stored on S3. -func (dm *Manager) doCleanUpRoutine() { - cnt := dm.CleanUpMeta() +func (sm *Manager) doCleanUpRoutine() { + cnt := sm.CleanUpMeta() if cnt != 0 { logutil.BgLogger().Info("clean up nodes in framework meta since nodes shutdown", zap.Int("cnt", cnt)) } - tasks, err := dm.taskMgr.GetTasksInStates( - dm.ctx, + tasks, err := sm.taskMgr.GetTasksInStates( + sm.ctx, proto.TaskStateFailed, proto.TaskStateReverted, proto.TaskStateSucceed, @@ -317,8 +317,8 @@ func (dm *Manager) doCleanUpRoutine() { if len(tasks) == 0 { return } - logutil.Logger(dm.ctx).Info("cleanUp routine start") - err = dm.cleanUpFinishedTasks(tasks) + logutil.Logger(sm.ctx).Info("cleanUp routine start") + err = sm.cleanUpFinishedTasks(tasks) if err != nil { logutil.BgLogger().Warn("cleanUp routine failed", zap.Error(err)) return @@ -326,19 +326,19 @@ func (dm *Manager) doCleanUpRoutine() { failpoint.Inject("WaitCleanUpFinished", func() { WaitCleanUpFinished <- struct{}{} }) - logutil.Logger(dm.ctx).Info("cleanUp routine success") + logutil.Logger(sm.ctx).Info("cleanUp routine success") } // CleanUpMeta clean up old node info in dist_framework_meta table. -func (dm *Manager) CleanUpMeta() int { +func (sm *Manager) CleanUpMeta() int { // Safe to discard errors since this function can be called at regular intervals. - serverInfos, err := GenerateTaskExecutorNodes(dm.ctx) + serverInfos, err := GenerateTaskExecutorNodes(sm.ctx) if err != nil { logutil.BgLogger().Warn("generate task executor nodes met error") return 0 } - oldNodes, err := dm.taskMgr.GetAllNodes(dm.ctx) + oldNodes, err := sm.taskMgr.GetAllNodes(sm.ctx) if err != nil { logutil.BgLogger().Warn("get all nodes met error") return 0 @@ -354,7 +354,7 @@ func (dm *Manager) CleanUpMeta() int { return 0 } logutil.BgLogger().Info("start to clean up dist_framework_meta") - err = dm.taskMgr.CleanUpMeta(dm.ctx, cleanNodes) + err = sm.taskMgr.CleanUpMeta(sm.ctx, cleanNodes) if err != nil { logutil.BgLogger().Warn("clean up dist_framework_meta met error") return 0 @@ -362,14 +362,14 @@ func (dm *Manager) CleanUpMeta() int { return len(cleanNodes) } -func (dm *Manager) cleanUpFinishedTasks(tasks []*proto.Task) error { +func (sm *Manager) cleanUpFinishedTasks(tasks []*proto.Task) error { cleanedTasks := make([]*proto.Task, 0) var firstErr error for _, task := range tasks { - cleanUpFactory := getDispatcherCleanUpFactory(task.Type) + cleanUpFactory := getSchedulerCleanUpFactory(task.Type) if cleanUpFactory != nil { cleanUp := cleanUpFactory() - err := cleanUp.CleanUp(dm.ctx, task) + err := cleanUp.CleanUp(sm.ctx, task) if err != nil { firstErr = err break @@ -384,10 +384,10 @@ func (dm *Manager) cleanUpFinishedTasks(tasks []*proto.Task) error { logutil.BgLogger().Warn("cleanUp routine failed", zap.Error(errors.Trace(firstErr))) } - return dm.taskMgr.TransferTasks2History(dm.ctx, cleanedTasks) + return sm.taskMgr.TransferTasks2History(sm.ctx, cleanedTasks) } -// MockDispatcher mock one dispatcher for one task, only used for tests. -func (dm *Manager) MockDispatcher(task *proto.Task) *BaseDispatcher { - return NewBaseDispatcher(dm.ctx, dm.taskMgr, dm.serverID, task) +// MockScheduler mock one scheduler for one task, only used for tests. +func (sm *Manager) MockScheduler(task *proto.Task) *BaseScheduler { + return NewBaseScheduler(sm.ctx, sm.taskMgr, sm.serverID, task) } diff --git a/pkg/disttask/framework/dispatcher/dispatcher_manager_test.go b/pkg/disttask/framework/scheduler/scheduler_manager_test.go similarity index 87% rename from pkg/disttask/framework/dispatcher/dispatcher_manager_test.go rename to pkg/disttask/framework/scheduler/scheduler_manager_test.go index dfec0c8127368..8d45536ebec4e 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher_manager_test.go +++ b/pkg/disttask/framework/scheduler/scheduler_manager_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher_test +package scheduler_test import ( "context" @@ -38,13 +38,13 @@ func TestCleanUpRoutine(t *testing.T) { ctrl := gomock.NewController(t) defer ctrl.Finish() ctx := context.Background() - ctx = util.WithInternalSourceType(ctx, "dispatcher_manager") + ctx = util.WithInternalSourceType(ctx, "scheduler_manager") mockCleanupRountine := mock.NewMockCleanUpRoutine(ctrl) - dsp, mgr := MockDispatcherManager(t, ctrl, pool, getNumberExampleDispatcherExt(ctrl), mockCleanupRountine) + sch, mgr := MockSchedulerManager(t, ctrl, pool, getNumberExampleSchedulerExt(ctrl), mockCleanupRountine) mockCleanupRountine.EXPECT().CleanUp(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - dsp.Start() - defer dsp.Stop() + sch.Start() + defer sch.Stop() require.NoError(t, mgr.StartManager(ctx, ":4000", "background")) taskID, err := mgr.CreateTask(ctx, "test", proto.TaskTypeExample, 1, nil) require.NoError(t, err) @@ -74,7 +74,7 @@ func TestCleanUpRoutine(t *testing.T) { err = mgr.UpdateSubtaskStateAndError(ctx, ":4000", int64(i), proto.TaskStateSucceed, nil) require.NoError(t, err) } - dsp.DoCleanUpRoutine() + sch.DoCleanUpRoutine() require.Eventually(t, func() bool { tasks, err := mgr.GetTasksFromHistoryInStates(ctx, proto.TaskStateSucceed) require.NoError(t, err) @@ -93,19 +93,19 @@ func TestCleanUpMeta(t *testing.T) { defer ctrl.Finish() mockTaskMgr := mock.NewMockTaskManager(ctrl) mockCleanupRountine := mock.NewMockCleanUpRoutine(ctrl) - dspMgr := MockDispatcherManagerWithMockTaskMgr(t, ctrl, pool, mockTaskMgr, getNumberExampleDispatcherExt(ctrl), mockCleanupRountine) + schMgr := MockSchedulerManagerWithMockTaskMgr(t, ctrl, pool, mockTaskMgr, getNumberExampleSchedulerExt(ctrl), mockCleanupRountine) mockTaskMgr.EXPECT().GetAllNodes(gomock.Any()).Return([]string{":4000", ":4001"}, nil) mockTaskMgr.EXPECT().CleanUpMeta(gomock.Any(), gomock.Any()).Return(nil) mockCleanupRountine.EXPECT().CleanUp(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - require.Equal(t, 1, dspMgr.CleanUpMeta()) + require.Equal(t, 1, schMgr.CleanUpMeta()) mockTaskMgr.EXPECT().GetAllNodes(gomock.Any()).Return([]string{":4000"}, nil) mockCleanupRountine.EXPECT().CleanUp(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - require.Equal(t, 0, dspMgr.CleanUpMeta()) + require.Equal(t, 0, schMgr.CleanUpMeta()) mockTaskMgr.EXPECT().GetAllNodes(gomock.Any()).Return([]string{":4000", ":4001", ":4003"}, nil) mockTaskMgr.EXPECT().CleanUpMeta(gomock.Any(), gomock.Any()).Return(nil) mockCleanupRountine.EXPECT().CleanUp(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - require.Equal(t, 2, dspMgr.CleanUpMeta()) + require.Equal(t, 2, schMgr.CleanUpMeta()) } diff --git a/pkg/disttask/framework/dispatcher/dispatcher_test.go b/pkg/disttask/framework/scheduler/scheduler_test.go similarity index 74% rename from pkg/disttask/framework/dispatcher/dispatcher_test.go rename to pkg/disttask/framework/scheduler/scheduler_test.go index 42115628194ac..7ce6c8c948bec 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher_test.go +++ b/pkg/disttask/framework/scheduler/scheduler_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher_test +package scheduler_test import ( "context" @@ -26,10 +26,10 @@ import ( "github.com/ngaut/pools" "github.com/pingcap/errors" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" - mockDispatch "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mock" "github.com/pingcap/tidb/pkg/disttask/framework/mock" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" + mockDispatch "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mock" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/framework/testutil" "github.com/pingcap/tidb/pkg/domain/infosync" @@ -49,41 +49,41 @@ const ( var mockedAllServerInfos = []*infosync.ServerInfo{} -func getTestDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( +func getTestSchedulerExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { return mockedAllServerInfos, true, nil }, ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( func(task *proto.Task) proto.Step { return proto.StepDone }, ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, _ *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, _ *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { return nil, nil }, ).AnyTimes() - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - return mockDispatcher + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + return mockScheduler } -func getNumberExampleDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( +func getNumberExampleSchedulerExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( func(ctx context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - serverInfo, err := dispatcher.GenerateTaskExecutorNodes(ctx) + serverInfo, err := scheduler.GenerateTaskExecutorNodes(ctx) return serverInfo, true, err }, ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( func(task *proto.Task) proto.Step { switch task.Step { case proto.StepInit: @@ -93,8 +93,8 @@ func getNumberExampleDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension } }, ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { switch task.Step { case proto.StepInit: for i := 0; i < subtaskCnt; i++ { @@ -111,40 +111,40 @@ func getNumberExampleDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension }, ).AnyTimes() - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - return mockDispatcher + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + return mockScheduler } -func MockDispatcherManager(t *testing.T, ctrl *gomock.Controller, pool *pools.ResourcePool, ext dispatcher.Extension, cleanUp dispatcher.CleanUpRoutine) (*dispatcher.Manager, *storage.TaskManager) { +func MockSchedulerManager(t *testing.T, ctrl *gomock.Controller, pool *pools.ResourcePool, ext scheduler.Extension, cleanUp scheduler.CleanUpRoutine) (*scheduler.Manager, *storage.TaskManager) { ctx := context.WithValue(context.Background(), "etcd", true) mgr := storage.NewTaskManager(pool) storage.SetTaskManager(mgr) - dsp, err := dispatcher.NewManager(util.WithInternalSourceType(ctx, "dispatcher"), mgr, "host:port") + sch, err := scheduler.NewManager(util.WithInternalSourceType(ctx, "scheduler"), mgr, "host:port") require.NoError(t, err) - dispatcher.RegisterDispatcherFactory(proto.TaskTypeExample, - func(ctx context.Context, taskMgr dispatcher.TaskManager, serverID string, task *proto.Task) dispatcher.Dispatcher { - mockDispatcher := dsp.MockDispatcher(task) - mockDispatcher.Extension = ext - return mockDispatcher + scheduler.RegisterSchedulerFactory(proto.TaskTypeExample, + func(ctx context.Context, taskMgr scheduler.TaskManager, serverID string, task *proto.Task) scheduler.Scheduler { + mockScheduler := sch.MockScheduler(task) + mockScheduler.Extension = ext + return mockScheduler }) - return dsp, mgr + return sch, mgr } -func MockDispatcherManagerWithMockTaskMgr(t *testing.T, ctrl *gomock.Controller, pool *pools.ResourcePool, taskMgr *mock.MockTaskManager, ext dispatcher.Extension, cleanUp dispatcher.CleanUpRoutine) *dispatcher.Manager { +func MockSchedulerManagerWithMockTaskMgr(t *testing.T, ctrl *gomock.Controller, pool *pools.ResourcePool, taskMgr *mock.MockTaskManager, ext scheduler.Extension, cleanUp scheduler.CleanUpRoutine) *scheduler.Manager { ctx := context.WithValue(context.Background(), "etcd", true) - dsp, err := dispatcher.NewManager(util.WithInternalSourceType(ctx, "dispatcher"), taskMgr, "host:port") + sch, err := scheduler.NewManager(util.WithInternalSourceType(ctx, "scheduler"), taskMgr, "host:port") require.NoError(t, err) - dispatcher.RegisterDispatcherFactory(proto.TaskTypeExample, - func(ctx context.Context, taskMgr dispatcher.TaskManager, serverID string, task *proto.Task) dispatcher.Dispatcher { - mockDispatcher := dsp.MockDispatcher(task) - mockDispatcher.Extension = ext - return mockDispatcher + scheduler.RegisterSchedulerFactory(proto.TaskTypeExample, + func(ctx context.Context, taskMgr scheduler.TaskManager, serverID string, task *proto.Task) scheduler.Scheduler { + mockScheduler := sch.MockScheduler(task) + mockScheduler.Extension = ext + return mockScheduler }) - dispatcher.RegisterDispatcherCleanUpFactory(proto.TaskTypeExample, - func() dispatcher.CleanUpRoutine { + scheduler.RegisterSchedulerCleanUpFactory(proto.TaskTypeExample, + func() scheduler.CleanUpRoutine { return cleanUp }) - return dsp + return sch } func deleteTasks(t *testing.T, store kv.Storage, taskID int64) { @@ -154,7 +154,7 @@ func deleteTasks(t *testing.T, store kv.Storage, taskID int64) { func TestGetInstance(t *testing.T) { ctx := context.Background() - ctx = util.WithInternalSourceType(ctx, "dispatcher") + ctx = util.WithInternalSourceType(ctx, "scheduler") store := testkit.CreateMockStore(t) gtk := testkit.NewTestKit(t, store) @@ -164,13 +164,13 @@ func TestGetInstance(t *testing.T) { defer pool.Close() ctrl := gomock.NewController(t) defer ctrl.Finish() - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockTaskExecutorNodes", "return()")) - dspManager, mgr := MockDispatcherManager(t, ctrl, pool, getTestDispatcherExt(ctrl), nil) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockTaskExecutorNodes", "return()")) + schManager, mgr := MockSchedulerManager(t, ctrl, pool, getTestSchedulerExt(ctrl), nil) // test no server task := &proto.Task{ID: 1, Type: proto.TaskTypeExample} - dsp := dspManager.MockDispatcher(task) - dsp.Extension = getTestDispatcherExt(ctrl) - instanceIDs, err := dsp.GetAllTaskExecutorIDs(ctx, task) + sch := schManager.MockScheduler(task) + sch.Extension = getTestSchedulerExt(ctrl) + instanceIDs, err := sch.GetAllTaskExecutorIDs(ctx, task) require.Lenf(t, instanceIDs, 0, "GetAllTaskExecutorIDs when there's no subtask") require.NoError(t, err) @@ -180,7 +180,7 @@ func TestGetInstance(t *testing.T) { uuids := []string{"ddl_id_1", "ddl_id_2"} serverIDs := []string{"10.123.124.10:32457", "[ABCD:EF01:2345:6789:ABCD:EF01:2345:6789]:65535"} - dispatcher.MockServerInfo = []*infosync.ServerInfo{ + scheduler.MockServerInfo = []*infosync.ServerInfo{ { ID: uuids[0], IP: "10.123.124.10", @@ -192,7 +192,7 @@ func TestGetInstance(t *testing.T) { Port: 65535, }, } - instanceIDs, err = dsp.GetAllTaskExecutorIDs(ctx, task) + instanceIDs, err = sch.GetAllTaskExecutorIDs(ctx, task) require.Lenf(t, instanceIDs, 0, "GetAllTaskExecutorIDs") require.NoError(t, err) @@ -204,7 +204,7 @@ func TestGetInstance(t *testing.T) { ExecID: serverIDs[1], } testutil.CreateSubTask(t, mgr, task.ID, proto.StepInit, subtask.ExecID, nil, subtask.Type, 11, true) - instanceIDs, err = dsp.GetAllTaskExecutorIDs(ctx, task) + instanceIDs, err = sch.GetAllTaskExecutorIDs(ctx, task) require.NoError(t, err) require.Equal(t, []string{serverIDs[1]}, instanceIDs) // server ids: uuid0, uuid1 @@ -215,11 +215,11 @@ func TestGetInstance(t *testing.T) { ExecID: serverIDs[0], } testutil.CreateSubTask(t, mgr, task.ID, proto.StepInit, subtask.ExecID, nil, subtask.Type, 11, true) - instanceIDs, err = dsp.GetAllTaskExecutorIDs(ctx, task) + instanceIDs, err = sch.GetAllTaskExecutorIDs(ctx, task) require.NoError(t, err) require.Len(t, instanceIDs, len(serverIDs)) require.ElementsMatch(t, instanceIDs, serverIDs) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockTaskExecutorNodes")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockTaskExecutorNodes")) } func TestTaskFailInManager(t *testing.T) { @@ -234,15 +234,15 @@ func TestTaskFailInManager(t *testing.T) { ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "handle_test") - mockDispatcher := mock.NewMockDispatcher(ctrl) - mockDispatcher.EXPECT().Init().Return(errors.New("mock dispatcher init error")) - dspManager, mgr := MockDispatcherManager(t, ctrl, pool, getTestDispatcherExt(ctrl), nil) - dispatcher.RegisterDispatcherFactory(proto.TaskTypeExample, - func(ctx context.Context, taskMgr dispatcher.TaskManager, serverID string, task *proto.Task) dispatcher.Dispatcher { - return mockDispatcher + mockScheduler := mock.NewMockScheduler(ctrl) + mockScheduler.EXPECT().Init().Return(errors.New("mock scheduler init error")) + schManager, mgr := MockSchedulerManager(t, ctrl, pool, getTestSchedulerExt(ctrl), nil) + scheduler.RegisterSchedulerFactory(proto.TaskTypeExample, + func(ctx context.Context, taskMgr scheduler.TaskManager, serverID string, task *proto.Task) scheduler.Scheduler { + return mockScheduler }) - dspManager.Start() - defer dspManager.Stop() + schManager.Start() + defer schManager.Stop() // unknown task type taskID, err := mgr.CreateTask(ctx, "test", "test-type", 1, nil) @@ -254,14 +254,14 @@ func TestTaskFailInManager(t *testing.T) { strings.Contains(task.Error.Error(), "unknown task type") }, time.Second*10, time.Millisecond*300) - // dispatcher init error + // scheduler init error taskID, err = mgr.CreateTask(ctx, "test2", proto.TaskTypeExample, 1, nil) require.NoError(t, err) require.Eventually(t, func() bool { task, err := mgr.GetTaskByID(ctx, taskID) require.NoError(t, err) return task.State == proto.TaskStateFailed && - strings.Contains(task.Error.Error(), "mock dispatcher init error") + strings.Contains(task.Error.Error(), "mock scheduler init error") }, time.Second*10, time.Millisecond*300) } @@ -288,12 +288,12 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, defer ctrl.Finish() ctx := context.Background() - ctx = util.WithInternalSourceType(ctx, "dispatcher") + ctx = util.WithInternalSourceType(ctx, "scheduler") - dsp, mgr := MockDispatcherManager(t, ctrl, pool, getNumberExampleDispatcherExt(ctrl), nil) - dsp.Start() + sch, mgr := MockSchedulerManager(t, ctrl, pool, getNumberExampleSchedulerExt(ctrl), nil) + sch.Start() defer func() { - dsp.Stop() + sch.Stop() // make data race happy if taskCnt == 1 { proto.MaxConcurrentTask = originalConcurrency @@ -306,7 +306,7 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, cnt := 60 checkGetRunningTaskCnt := func(expected int) { require.Eventually(t, func() bool { - return dsp.GetRunningTaskCnt() == expected + return sch.GetRunningTaskCnt() == expected }, time.Second, 50*time.Millisecond) } @@ -350,7 +350,7 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, checkGetRunningTaskCnt(taskCnt) // Clean the task. deleteTasks(t, store, taskID) - dsp.DelRunningTask(taskID) + sch.DelRunningTask(taskID) } // test DetectTaskLoop @@ -507,14 +507,14 @@ func TestVerifyTaskStateTransform(t *testing.T) { {proto.TaskStateCanceled, proto.TaskStateRunning, false}, } for _, tc := range testCases { - require.Equal(t, tc.expect, dispatcher.VerifyTaskStateTransform(tc.oldState, tc.newState)) + require.Equal(t, tc.expect, scheduler.VerifyTaskStateTransform(tc.oldState, tc.newState)) } } func TestIsCancelledErr(t *testing.T) { - require.False(t, dispatcher.IsCancelledErr(errors.New("some err"))) - require.False(t, dispatcher.IsCancelledErr(context.Canceled)) - require.True(t, dispatcher.IsCancelledErr(errors.New("cancelled by user"))) + require.False(t, scheduler.IsCancelledErr(errors.New("some err"))) + require.False(t, scheduler.IsCancelledErr(context.Canceled)) + require.True(t, scheduler.IsCancelledErr(errors.New("cancelled by user"))) } func TestManagerDispatchLoop(t *testing.T) { @@ -525,7 +525,7 @@ func TestManagerDispatchLoop(t *testing.T) { }) ctrl := gomock.NewController(t) defer ctrl.Finish() - mockDispatcher := mock.NewMockDispatcher(ctrl) + mockScheduler := mock.NewMockScheduler(ctrl) _ = testkit.CreateMockStore(t) require.Eventually(t, func() bool { @@ -534,12 +534,12 @@ func TestManagerDispatchLoop(t *testing.T) { }, 10*time.Second, 100*time.Millisecond) ctx := context.Background() - ctx = util.WithInternalSourceType(ctx, "dispatcher") + ctx = util.WithInternalSourceType(ctx, "scheduler") taskMgr, err := storage.GetTaskManager() require.NoError(t, err) require.NotNil(t, taskMgr) - // in this test, we only test dispatcher manager, so we add a subtask takes 16 + // in this test, we only test scheduler manager, so we add a subtask takes 16 // slots to avoid reserve by slots, and make sure below test cases works. serverInfos, err := infosync.GetAllServerInfo(ctx) require.NoError(t, err) @@ -553,12 +553,12 @@ func TestManagerDispatchLoop(t *testing.T) { waitChannels[i] = make(chan struct{}) } var counter atomic.Int32 - dispatcher.RegisterDispatcherFactory(proto.TaskTypeExample, - func(ctx context.Context, taskMgr dispatcher.TaskManager, serverID string, task *proto.Task) dispatcher.Dispatcher { + scheduler.RegisterSchedulerFactory(proto.TaskTypeExample, + func(ctx context.Context, taskMgr scheduler.TaskManager, serverID string, task *proto.Task) scheduler.Scheduler { idx := counter.Load() - mockDispatcher = mock.NewMockDispatcher(ctrl) - mockDispatcher.EXPECT().Init().Return(nil) - mockDispatcher.EXPECT().ExecuteTask().Do(func() { + mockScheduler = mock.NewMockScheduler(ctrl) + mockScheduler.EXPECT().Init().Return(nil) + mockScheduler.EXPECT().ExecuteTask().Do(func() { require.NoError(t, taskMgr.WithNewSession(func(se sessionctx.Context) error { _, err := storage.ExecSQL(ctx, se, "update mysql.tidb_global_task set state=%?, step=%? where id=%?", proto.TaskStateRunning, proto.StepOne, task.ID) @@ -571,9 +571,9 @@ func TestManagerDispatchLoop(t *testing.T) { return err })) }) - mockDispatcher.EXPECT().Close() + mockScheduler.EXPECT().Close() counter.Add(1) - return mockDispatcher + return mockScheduler }, ) for i := 0; i < len(concurrencies); i++ { diff --git a/pkg/disttask/framework/dispatcher/slots.go b/pkg/disttask/framework/scheduler/slots.go similarity index 95% rename from pkg/disttask/framework/dispatcher/slots.go rename to pkg/disttask/framework/scheduler/slots.go index 72fe68ef72edc..01de011de354b 100644 --- a/pkg/disttask/framework/dispatcher/slots.go +++ b/pkg/disttask/framework/scheduler/slots.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "context" @@ -39,7 +39,7 @@ type taskStripes struct { // all nodes managed by dist framework are isomorphic. // Stripes reserved for a task defines the maximum resource that a task can use // but the task might not use all the resources. To maximize the resource utilization, -// we will try to dispatch as many tasks as possible depends on the used slots +// we will try to schedule as many tasks as possible depends on the used slots // on each node and the minimum resource required by the tasks, and in this case, // we don't consider task order. // @@ -61,13 +61,13 @@ type slotManager struct { // map of reservedStripes for fast delete task2Index map[int64]int // represents the number of slots reserved by task on each node, the execID - // is only used for reserve minimum resource when starting dispatcher, the + // is only used for reserve minimum resource when starting scheduler, the // subtasks may or may not be scheduled on this node. reservedSlots map[string]int // represents the number of slots taken by task on each node // on some cases it might be larger than capacity: // current step of higher priority task A has little subtasks, so we start - // to dispatch lower priority task, but next step of A has many subtasks. + // to schedule lower priority task, but next step of A has many subtasks. // once initialized, the length of usedSlots should be equal to number of nodes // managed by dist framework. usedSlots map[string]int @@ -109,7 +109,7 @@ func (sm *slotManager) update(ctx context.Context, taskMgr TaskManager) error { // else if the resource is reserved by stripes, it returns "". // as usedSlots is updated asynchronously, it might return false even if there // are enough resources, or return true on resource shortage when some task -// dispatched subtasks. +// scheduled subtasks. func (sm *slotManager) canReserve(task *proto.Task) (execID string, ok bool) { sm.mu.RLock() defer sm.mu.RUnlock() diff --git a/pkg/disttask/framework/dispatcher/slots_test.go b/pkg/disttask/framework/scheduler/slots_test.go similarity index 99% rename from pkg/disttask/framework/dispatcher/slots_test.go rename to pkg/disttask/framework/scheduler/slots_test.go index d3eb0162075cc..8363c7df381a9 100644 --- a/pkg/disttask/framework/dispatcher/slots_test.go +++ b/pkg/disttask/framework/scheduler/slots_test.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "context" diff --git a/pkg/disttask/framework/dispatcher/state_transform.go b/pkg/disttask/framework/scheduler/state_transform.go similarity index 99% rename from pkg/disttask/framework/dispatcher/state_transform.go rename to pkg/disttask/framework/scheduler/state_transform.go index 0c7ae1bca2847..1e3a39a2614b5 100644 --- a/pkg/disttask/framework/dispatcher/state_transform.go +++ b/pkg/disttask/framework/scheduler/state_transform.go @@ -12,7 +12,7 @@ // See the License for the specific language governing permissions and // limitations under the License. -package dispatcher +package scheduler import ( "github.com/pingcap/tidb/pkg/disttask/framework/proto" diff --git a/pkg/disttask/framework/storage/table_test.go b/pkg/disttask/framework/storage/table_test.go index bb451a87d432b..18e32cd3c1e5c 100644 --- a/pkg/disttask/framework/storage/table_test.go +++ b/pkg/disttask/framework/storage/table_test.go @@ -547,7 +547,7 @@ func TestBothTaskAndSubTaskTable(t *testing.T) { } func TestDistFrameworkMeta(t *testing.T) { - // to avoid inserted nodes be cleaned by dispatcher + // to avoid inserted nodes be cleaned by scheduler require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/domain/MockDisableDistTask", "return(true)")) defer func() { require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/domain/MockDisableDistTask")) diff --git a/pkg/disttask/framework/storage/task_table.go b/pkg/disttask/framework/storage/task_table.go index fb95aa4c5691a..de8aea4f4a70f 100644 --- a/pkg/disttask/framework/storage/task_table.go +++ b/pkg/disttask/framework/storage/task_table.go @@ -42,7 +42,8 @@ const ( defaultSubtaskKeepDays = 14 basicTaskColumns = `id, task_key, type, state, step, priority, concurrency, create_time` - taskColumns = basicTaskColumns + `, start_time, state_update_time, meta, dispatcher_id, error` + // TODO: dispatcher_id will update to scheduler_id later + taskColumns = basicTaskColumns + `, start_time, state_update_time, meta, dispatcher_id, error` // InsertTaskColumns is the columns used in insert task. InsertTaskColumns = `task_key, type, state, priority, concurrency, step, meta, create_time, start_time, state_update_time` @@ -134,7 +135,7 @@ func row2Task(r chunk.Row) *proto.Task { task.StartTime, _ = r.GetTime(8).GoTime(time.Local) task.StateUpdateTime, _ = r.GetTime(9).GoTime(time.Local) task.Meta = r.GetBytes(10) - task.DispatcherID = r.GetString(11) + task.SchedulerID = r.GetString(11) if !r.IsNull(12) { errBytes := r.GetBytes(12) stdErr := errors.Normalize("") @@ -233,7 +234,7 @@ func (*TaskManager) CreateTaskWithSession(ctx context.Context, se sessionctx.Con return taskID, nil } -// GetOneTask get a task from task table, it's used by dispatcher only. +// GetOneTask get a task from task table, it's used by scheduler only. func (stm *TaskManager) GetOneTask(ctx context.Context) (task *proto.Task, err error) { rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task where state = %? limit 1", proto.TaskStatePending) if err != nil { @@ -247,7 +248,7 @@ func (stm *TaskManager) GetOneTask(ctx context.Context) (task *proto.Task, err e return row2Task(rs[0]), nil } -// GetTopUnfinishedTasks implements the dispatcher.TaskManager interface. +// GetTopUnfinishedTasks implements the scheduler.TaskManager interface. func (stm *TaskManager) GetTopUnfinishedTasks(ctx context.Context) (task []*proto.Task, err error) { rs, err := stm.executeSQLWithNewSession(ctx, `select `+basicTaskColumns+` from mysql.tidb_global_task @@ -360,7 +361,7 @@ func (stm *TaskManager) GetTaskByKeyWithHistory(ctx context.Context, key string) return row2Task(rs[0]), nil } -// FailTask implements the dispatcher.TaskManager interface. +// FailTask implements the scheduler.TaskManager interface. func (stm *TaskManager) FailTask(ctx context.Context, taskID int64, currentState proto.TaskState, taskErr error) error { _, err := stm.executeSQLWithNewSession(ctx, `update mysql.tidb_global_task @@ -373,7 +374,7 @@ func (stm *TaskManager) FailTask(ctx context.Context, taskID int64, currentState return err } -// GetUsedSlotsOnNodes implements the dispatcher.TaskManager interface. +// GetUsedSlotsOnNodes implements the scheduler.TaskManager interface. func (stm *TaskManager) GetUsedSlotsOnNodes(ctx context.Context) (map[string]int, error) { // concurrency of subtasks of some step is the same, we use max(concurrency) // to make group by works. @@ -793,11 +794,11 @@ func (stm *TaskManager) UpdateTaskAndAddSubTasks(ctx context.Context, task *prot _, err := ExecSQL(ctx, se, "update mysql.tidb_global_task "+ "set state = %?, dispatcher_id = %?, step = %?, concurrency = %?, meta = %?, error = %?, state_update_time = CURRENT_TIMESTAMP()"+ "where id = %? and state = %?", - task.State, task.DispatcherID, task.Step, task.Concurrency, task.Meta, serializeErr(task.Error), task.ID, prevState) + task.State, task.SchedulerID, task.Step, task.Concurrency, task.Meta, serializeErr(task.Error), task.ID, prevState) if err != nil { return err } - // When AffectedRows == 0, means other admin command have changed the task state, it's illegal to dispatch subtasks. + // When AffectedRows == 0, means other admin command have changed the task state, it's illegal to schedule subtasks. if se.GetSessionVars().StmtCtx.AffectedRows() == 0 { if !intest.InTest { // task state have changed by other admin command @@ -806,7 +807,7 @@ func (stm *TaskManager) UpdateTaskAndAddSubTasks(ctx context.Context, task *prot } // TODO: remove it, when OnNextSubtasksBatch returns subtasks, just insert subtasks without updating tidb_global_task. // Currently the business running on distributed task framework will update proto.Task in OnNextSubtasksBatch. - // So when dispatching subtasks, framework needs to update task and insert subtasks in one Txn. + // So when scheduling subtasks, framework needs to update task and insert subtasks in one Txn. // // In future, it's needed to restrict changes of task in OnNextSubtasksBatch. // If OnNextSubtasksBatch won't update any fields in proto.Task, we can insert subtasks only. @@ -1046,7 +1047,7 @@ func (stm *TaskManager) TransferTasks2History(ctx context.Context, tasks []*prot } } if err := sqlescape.FormatSQL(insertSQL, "(%?, %?, %?, %?, %?, %?, %?, %?, %?, %?, %?, %?)", - task.ID, task.Key, task.Type, task.DispatcherID, + task.ID, task.Key, task.Type, task.SchedulerID, task.State, task.Priority, task.StartTime, task.StateUpdateTime, task.Meta, task.Concurrency, task.Step, serializeErr(task.Error)); err != nil { return err @@ -1074,7 +1075,7 @@ func (stm *TaskManager) TransferTasks2History(ctx context.Context, tasks []*prot }) } -// GetManagedNodes implements dispatcher.TaskManager interface. +// GetManagedNodes implements scheduler.TaskManager interface. func (stm *TaskManager) GetManagedNodes(ctx context.Context) ([]string, error) { rs, err := stm.executeSQLWithNewSession(ctx, ` select host, role diff --git a/pkg/disttask/framework/taskexecutor/BUILD.bazel b/pkg/disttask/framework/taskexecutor/BUILD.bazel index fc92fd870d4d7..d2e479c6ddfbc 100644 --- a/pkg/disttask/framework/taskexecutor/BUILD.bazel +++ b/pkg/disttask/framework/taskexecutor/BUILD.bazel @@ -14,9 +14,9 @@ go_library( "//br/pkg/lightning/common", "//br/pkg/lightning/log", "//pkg/config", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/handle", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/taskexecutor/execute", "//pkg/domain/infosync", diff --git a/pkg/disttask/framework/taskexecutor/manager.go b/pkg/disttask/framework/taskexecutor/manager.go index f25d613827260..14790f755fb24 100644 --- a/pkg/disttask/framework/taskexecutor/manager.go +++ b/pkg/disttask/framework/taskexecutor/manager.go @@ -35,7 +35,7 @@ import ( var ( executorPoolSize int32 = 4 - // same as dispatcher + // same as scheduler checkTime = 300 * time.Millisecond recoverMetaInterval = 90 * time.Second retrySQLTimes = 30 @@ -264,7 +264,7 @@ func (m *Manager) onPausingTasks(tasks []*proto.Task) error { // recoverMetaLoop inits and recovers dist_framework_meta for the tidb node running the taskExecutor manager. // This is necessary when the TiDB node experiences a prolonged network partition -// and the dispatcher deletes `dist_framework_meta`. +// and the scheduler deletes `dist_framework_meta`. // When the TiDB node recovers from the network partition, // we need to re-insert the metadata. func (m *Manager) recoverMetaLoop() { diff --git a/pkg/disttask/framework/taskexecutor/task_executor.go b/pkg/disttask/framework/taskexecutor/task_executor.go index 8f29fe2044865..2feaa8bc65129 100644 --- a/pkg/disttask/framework/taskexecutor/task_executor.go +++ b/pkg/disttask/framework/taskexecutor/task_executor.go @@ -23,9 +23,9 @@ import ( "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/common" "github.com/pingcap/tidb/br/pkg/lightning/log" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/handle" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/execute" "github.com/pingcap/tidb/pkg/domain/infosync" @@ -542,8 +542,8 @@ func (s *BaseTaskExecutor) startSubtaskAndUpdateState(ctx context.Context, subta func (s *BaseTaskExecutor) updateSubtaskStateAndErrorImpl(ctx context.Context, tidbID string, subtaskID int64, state proto.TaskState, subTaskErr error) { // retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes logger := logutil.Logger(s.logCtx) - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - err := handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + err := handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, s.taskTable.UpdateSubtaskStateAndError(ctx, tidbID, subtaskID, state, subTaskErr) }, @@ -556,8 +556,8 @@ func (s *BaseTaskExecutor) updateSubtaskStateAndErrorImpl(ctx context.Context, t func (s *BaseTaskExecutor) startSubtask(ctx context.Context, subtaskID int64) { // retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes logger := logutil.Logger(s.logCtx) - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - err := handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + err := handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, s.taskTable.StartSubtask(ctx, subtaskID) }, @@ -569,8 +569,8 @@ func (s *BaseTaskExecutor) startSubtask(ctx context.Context, subtaskID int64) { func (s *BaseTaskExecutor) finishSubtask(ctx context.Context, subtask *proto.Subtask) { logger := logutil.Logger(s.logCtx) - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - err := handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + err := handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, s.taskTable.FinishSubtask(ctx, subtask.ExecID, subtask.ID, subtask.Meta) }, @@ -625,8 +625,8 @@ func (s *BaseTaskExecutor) markSubTaskCanceledOrFailed(ctx context.Context, subt func (s *BaseTaskExecutor) updateErrorToSubtask(ctx context.Context, taskID int64, err error) error { logger := logutil.Logger(s.logCtx) - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - err1 := handle.RunWithRetry(s.logCtx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + err1 := handle.RunWithRetry(s.logCtx, scheduler.RetrySQLTimes, backoffer, logger, func(_ context.Context) (bool, error) { return true, s.taskTable.UpdateErrorToSubtask(ctx, s.id, taskID, err) }, diff --git a/pkg/disttask/framework/testutil/BUILD.bazel b/pkg/disttask/framework/testutil/BUILD.bazel index ed5013dfb4809..f5ab5b4bb5436 100644 --- a/pkg/disttask/framework/testutil/BUILD.bazel +++ b/pkg/disttask/framework/testutil/BUILD.bazel @@ -4,20 +4,20 @@ go_library( name = "testutil", srcs = [ "context.go", - "dispatcher_util.go", "disttest_util.go", "executor_util.go", + "scheduler_util.go", "table_util.go", "task_util.go", ], importpath = "github.com/pingcap/tidb/pkg/disttask/framework/testutil", visibility = ["//visibility:public"], deps = [ - "//pkg/disttask/framework/dispatcher", - "//pkg/disttask/framework/dispatcher/mock", "//pkg/disttask/framework/mock", "//pkg/disttask/framework/mock/execute", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", + "//pkg/disttask/framework/scheduler/mock", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/taskexecutor", "//pkg/domain/infosync", diff --git a/pkg/disttask/framework/testutil/context.go b/pkg/disttask/framework/testutil/context.go index aa13e681bd3c5..7b440f328d7d0 100644 --- a/pkg/disttask/framework/testutil/context.go +++ b/pkg/disttask/framework/testutil/context.go @@ -40,7 +40,7 @@ func InitTestContext(t *testing.T, nodeNum int) (context.Context, *gomock.Contro ctrl := gomock.NewController(t) defer ctrl.Finish() ctx := context.Background() - ctx = util.WithInternalSourceType(ctx, "dispatcher") + ctx = util.WithInternalSourceType(ctx, "scheduler") return ctx, ctrl, &TestContext{}, testkit.NewDistExecutionContext(t, nodeNum) } diff --git a/pkg/disttask/framework/testutil/dispatcher_util.go b/pkg/disttask/framework/testutil/dispatcher_util.go deleted file mode 100644 index 6d955ee64bf4b..0000000000000 --- a/pkg/disttask/framework/testutil/dispatcher_util.go +++ /dev/null @@ -1,298 +0,0 @@ -// Copyright 2023 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package testutil - -import ( - "context" - "errors" - - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" - mockDispatch "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mock" - "github.com/pingcap/tidb/pkg/disttask/framework/proto" - "github.com/pingcap/tidb/pkg/domain/infosync" - "go.uber.org/mock/gomock" -) - -// GetMockBasicDispatcherExt returns mock dispatcher.Extension with basic functionalities. -func GetMockBasicDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - return generateTaskExecutorNodes4Test() - }, - ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( - func(task *proto.Task) proto.Step { - switch task.Step { - case proto.StepInit: - return proto.StepOne - case proto.StepOne: - return proto.StepTwo - default: - return proto.StepDone - } - }, - ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if task.Step == proto.StepInit { - return [][]byte{ - []byte("task1"), - []byte("task2"), - []byte("task3"), - }, nil - } - if task.Step == proto.StepOne { - return [][]byte{ - []byte("task4"), - }, nil - } - return nil, nil - }, - ).AnyTimes() - - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - return mockDispatcher -} - -// GetMockHATestDispatcherExt returns mock dispatcher.Extension for HA testing with multiple steps. -func GetMockHATestDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - return generateTaskExecutorNodes4Test() - }, - ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( - func(task *proto.Task) proto.Step { - switch task.Step { - case proto.StepInit: - return proto.StepOne - case proto.StepOne: - return proto.StepTwo - default: - return proto.StepDone - } - }, - ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if task.Step == proto.StepInit { - return [][]byte{ - []byte("task1"), - []byte("task2"), - []byte("task3"), - []byte("task4"), - []byte("task5"), - []byte("task6"), - []byte("task7"), - []byte("task8"), - []byte("task9"), - []byte("task10"), - }, nil - } - if task.Step == proto.StepOne { - return [][]byte{ - []byte("task11"), - []byte("task12"), - []byte("task13"), - []byte("task14"), - []byte("task15"), - }, nil - } - return nil, nil - }, - ).AnyTimes() - - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - - return mockDispatcher -} - -func generateTaskExecutorNodes4Test() ([]*infosync.ServerInfo, bool, error) { - serverInfos := infosync.MockGlobalServerInfoManagerEntry.GetAllServerInfo() - if len(serverInfos) == 0 { - return nil, true, errors.New("not found instance") - } - - serverNodes := make([]*infosync.ServerInfo, 0, len(serverInfos)) - for _, serverInfo := range serverInfos { - serverNodes = append(serverNodes, serverInfo) - } - return serverNodes, true, nil -} - -// GetPlanNotRetryableErrDispatcherExt returns mock dispatcher.Extension which will generate non retryable error when planning. -func GetPlanNotRetryableErrDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - return generateTaskExecutorNodes4Test() - }, - ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(false).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( - func(task *proto.Task) proto.Step { - return proto.StepDone - }, - ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, _ *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - return nil, errors.New("not retryable err") - }, - ).AnyTimes() - - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - return mockDispatcher -} - -// GetPlanErrDispatcherExt returns mock dispatcher.Extension which will generate error when planning. -func GetPlanErrDispatcherExt(ctrl *gomock.Controller, testContext *TestContext) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - return generateTaskExecutorNodes4Test() - }, - ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( - func(task *proto.Task) proto.Step { - switch task.Step { - case proto.StepInit: - return proto.StepOne - case proto.StepOne: - return proto.StepTwo - default: - return proto.StepDone - } - }, - ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if task.Step == proto.StepInit { - if testContext.CallTime == 0 { - testContext.CallTime++ - return nil, errors.New("retryable err") - } - return [][]byte{ - []byte("task1"), - []byte("task2"), - []byte("task3"), - }, nil - } - if task.Step == proto.StepOne { - return [][]byte{ - []byte("task4"), - }, nil - } - return nil, nil - }, - ).AnyTimes() - - gomock.InOrder( - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(errors.New("not retryable err")), - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), - ) - return mockDispatcher -} - -// GetMockRollbackDispatcherExt returns mock dispatcher.Extension which will generate rollback subtasks. -func GetMockRollbackDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - return generateTaskExecutorNodes4Test() - }, - ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( - func(task *proto.Task) proto.Step { - switch task.Step { - case proto.StepInit: - return proto.StepOne - default: - return proto.StepDone - } - }, - ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if task.Step == proto.StepInit { - return [][]byte{ - []byte("task1"), - []byte("task2"), - []byte("task3"), - }, nil - } - return nil, nil - }, - ).AnyTimes() - - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - return mockDispatcher -} - -// GetMockDynamicDispatchExt returns mock dispatcher.Extension which will generate subtask in multiple batches. -func GetMockDynamicDispatchExt(ctrl *gomock.Controller) dispatcher.Extension { - mockDispatcher := mockDispatch.NewMockExtension(ctrl) - mockDispatcher.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() - mockDispatcher.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { - return generateTaskExecutorNodes4Test() - }, - ).AnyTimes() - mockDispatcher.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() - mockDispatcher.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( - func(task *proto.Task) proto.Step { - switch task.Step { - case proto.StepInit: - return proto.StepOne - case proto.StepOne: - return proto.StepTwo - default: - return proto.StepDone - } - }, - ).AnyTimes() - mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if task.Step == proto.StepInit { - return [][]byte{ - []byte("task"), - []byte("task"), - }, nil - } - - // step2 - if task.Step == proto.StepOne { - return [][]byte{ - []byte("task"), - }, nil - } - return nil, nil - }, - ).AnyTimes() - - mockDispatcher.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() - - return mockDispatcher -} diff --git a/pkg/disttask/framework/testutil/disttest_util.go b/pkg/disttask/framework/testutil/disttest_util.go index ca4e0d5b4a575..de6778cdf0252 100644 --- a/pkg/disttask/framework/testutil/disttest_util.go +++ b/pkg/disttask/framework/testutil/disttest_util.go @@ -22,10 +22,10 @@ import ( "time" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/mock" mockexecute "github.com/pingcap/tidb/pkg/disttask/framework/mock/execute" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor" "github.com/stretchr/testify/require" @@ -33,7 +33,7 @@ import ( ) // RegisterTaskMeta initialize mock components for dist task. -func RegisterTaskMeta(t *testing.T, ctrl *gomock.Controller, dispatcherHandle dispatcher.Extension, testContext *TestContext, runSubtaskFn func(ctx context.Context, subtask *proto.Subtask) error) { +func RegisterTaskMeta(t *testing.T, ctrl *gomock.Controller, schedulerHandle scheduler.Extension, testContext *TestContext, runSubtaskFn func(ctx context.Context, subtask *proto.Subtask) error) { mockExtension := mock.NewMockExtension(ctrl) mockCleanupRountine := mock.NewMockCleanUpRoutine(ctrl) mockCleanupRountine.EXPECT().CleanUp(gomock.Any(), gomock.Any()).Return(nil).AnyTimes() @@ -57,24 +57,24 @@ func RegisterTaskMeta(t *testing.T, ctrl *gomock.Controller, dispatcherHandle di mockExtension.EXPECT().IsIdempotent(gomock.Any()).Return(true).AnyTimes() mockExtension.EXPECT().GetSubtaskExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockSubtaskExecutor, nil).AnyTimes() mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(false).AnyTimes() - registerTaskMetaInner(t, proto.TaskTypeExample, mockExtension, mockCleanupRountine, dispatcherHandle) + registerTaskMetaInner(t, proto.TaskTypeExample, mockExtension, mockCleanupRountine, schedulerHandle) } -func registerTaskMetaInner(t *testing.T, taskType proto.TaskType, mockExtension taskexecutor.Extension, mockCleanup dispatcher.CleanUpRoutine, dispatcherHandle dispatcher.Extension) { +func registerTaskMetaInner(t *testing.T, taskType proto.TaskType, mockExtension taskexecutor.Extension, mockCleanup scheduler.CleanUpRoutine, schedulerHandle scheduler.Extension) { t.Cleanup(func() { - dispatcher.ClearDispatcherFactory() - dispatcher.ClearDispatcherCleanUpFactory() + scheduler.ClearSchedulerFactory() + scheduler.ClearSchedulerCleanUpFactory() taskexecutor.ClearTaskExecutors() }) - dispatcher.RegisterDispatcherFactory(taskType, - func(ctx context.Context, taskMgr dispatcher.TaskManager, serverID string, task *proto.Task) dispatcher.Dispatcher { - baseDispatcher := dispatcher.NewBaseDispatcher(ctx, taskMgr, serverID, task) - baseDispatcher.Extension = dispatcherHandle - return baseDispatcher + scheduler.RegisterSchedulerFactory(taskType, + func(ctx context.Context, taskMgr scheduler.TaskManager, serverID string, task *proto.Task) scheduler.Scheduler { + baseScheduler := scheduler.NewBaseScheduler(ctx, taskMgr, serverID, task) + baseScheduler.Extension = schedulerHandle + return baseScheduler }) - dispatcher.RegisterDispatcherCleanUpFactory(taskType, - func() dispatcher.CleanUpRoutine { + scheduler.RegisterSchedulerCleanUpFactory(taskType, + func() scheduler.CleanUpRoutine { return mockCleanup }) @@ -88,7 +88,7 @@ func registerTaskMetaInner(t *testing.T, taskType proto.TaskType, mockExtension } // RegisterRollbackTaskMeta register rollback task meta. -func RegisterRollbackTaskMeta(t *testing.T, ctrl *gomock.Controller, mockDispatcher dispatcher.Extension, testContext *TestContext) { +func RegisterRollbackTaskMeta(t *testing.T, ctrl *gomock.Controller, mockScheduler scheduler.Extension, testContext *TestContext) { mockExtension := mock.NewMockExtension(ctrl) mockExecutor := mockexecute.NewMockSubtaskExecutor(ctrl) mockCleanupRountine := mock.NewMockCleanUpRoutine(ctrl) @@ -111,11 +111,11 @@ func RegisterRollbackTaskMeta(t *testing.T, ctrl *gomock.Controller, mockDispatc mockExtension.EXPECT().GetSubtaskExecutor(gomock.Any(), gomock.Any(), gomock.Any()).Return(mockExecutor, nil).AnyTimes() mockExtension.EXPECT().IsRetryableError(gomock.Any()).Return(false).AnyTimes() - registerTaskMetaInner(t, proto.TaskTypeExample, mockExtension, mockCleanupRountine, mockDispatcher) + registerTaskMetaInner(t, proto.TaskTypeExample, mockExtension, mockCleanupRountine, mockScheduler) testContext.RollbackCnt.Store(0) } -// DispatchTask dispatch one task. +// DispatchTask schedule one task. func DispatchTask(ctx context.Context, t *testing.T, taskKey string) *proto.Task { mgr, err := storage.GetTaskManager() require.NoError(t, err) @@ -146,7 +146,7 @@ func WaitTaskExit(ctx context.Context, t *testing.T, taskKey string) *proto.Task return task } -// DispatchTaskAndCheckSuccess dispatch one task and check if it is succeed. +// DispatchTaskAndCheckSuccess schedule one task and check if it is succeed. func DispatchTaskAndCheckSuccess(ctx context.Context, t *testing.T, taskKey string, testContext *TestContext, checkResultFn func(t *testing.T, testContext *TestContext)) { task := DispatchTask(ctx, t, taskKey) require.Equal(t, proto.TaskStateSucceed, task.State) @@ -163,7 +163,7 @@ func DispatchTaskAndCheckSuccess(ctx context.Context, t *testing.T, taskKey stri testContext.M = sync.Map{} } -// DispatchAndCancelTask dispatch one task then cancel it. +// DispatchAndCancelTask schedule one task then cancel it. func DispatchAndCancelTask(ctx context.Context, t *testing.T, taskKey string, testContext *TestContext) { require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunCancel", "1*return(1)")) defer func() { @@ -177,7 +177,7 @@ func DispatchAndCancelTask(ctx context.Context, t *testing.T, taskKey string, te }) } -// DispatchTaskAndCheckState dispatch one task and check the task state. +// DispatchTaskAndCheckState schedule one task and check the task state. func DispatchTaskAndCheckState(ctx context.Context, t *testing.T, taskKey string, testContext *TestContext, state proto.TaskState) { task := DispatchTask(ctx, t, taskKey) require.Equal(t, state, task.State) @@ -187,7 +187,7 @@ func DispatchTaskAndCheckState(ctx context.Context, t *testing.T, taskKey string }) } -// DispatchMultiTasksAndOneFail dispatches multiple tasks and force one task failed. +// DispatchMultiTasksAndOneFail schedulees multiple tasks and force one task failed. // TODO(ywqzzy): run tasks with multiple types. func DispatchMultiTasksAndOneFail(ctx context.Context, t *testing.T, num int, testContext *TestContext) { mgr, err := storage.GetTaskManager() diff --git a/pkg/disttask/framework/testutil/scheduler_util.go b/pkg/disttask/framework/testutil/scheduler_util.go new file mode 100644 index 0000000000000..ae633814f2a20 --- /dev/null +++ b/pkg/disttask/framework/testutil/scheduler_util.go @@ -0,0 +1,298 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package testutil + +import ( + "context" + "errors" + + "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" + mockDispatch "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mock" + "github.com/pingcap/tidb/pkg/domain/infosync" + "go.uber.org/mock/gomock" +) + +// GetMockBasicSchedulerExt returns mock scheduler.Extension with basic functionalities. +func GetMockBasicSchedulerExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + return generateTaskExecutorNodes4Test() + }, + ).AnyTimes() + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + func(task *proto.Task) proto.Step { + switch task.Step { + case proto.StepInit: + return proto.StepOne + case proto.StepOne: + return proto.StepTwo + default: + return proto.StepDone + } + }, + ).AnyTimes() + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + if task.Step == proto.StepInit { + return [][]byte{ + []byte("task1"), + []byte("task2"), + []byte("task3"), + }, nil + } + if task.Step == proto.StepOne { + return [][]byte{ + []byte("task4"), + }, nil + } + return nil, nil + }, + ).AnyTimes() + + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + return mockScheduler +} + +// GetMockHATestSchedulerExt returns mock scheduler.Extension for HA testing with multiple steps. +func GetMockHATestSchedulerExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + return generateTaskExecutorNodes4Test() + }, + ).AnyTimes() + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + func(task *proto.Task) proto.Step { + switch task.Step { + case proto.StepInit: + return proto.StepOne + case proto.StepOne: + return proto.StepTwo + default: + return proto.StepDone + } + }, + ).AnyTimes() + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + if task.Step == proto.StepInit { + return [][]byte{ + []byte("task1"), + []byte("task2"), + []byte("task3"), + []byte("task4"), + []byte("task5"), + []byte("task6"), + []byte("task7"), + []byte("task8"), + []byte("task9"), + []byte("task10"), + }, nil + } + if task.Step == proto.StepOne { + return [][]byte{ + []byte("task11"), + []byte("task12"), + []byte("task13"), + []byte("task14"), + []byte("task15"), + }, nil + } + return nil, nil + }, + ).AnyTimes() + + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + + return mockScheduler +} + +func generateTaskExecutorNodes4Test() ([]*infosync.ServerInfo, bool, error) { + serverInfos := infosync.MockGlobalServerInfoManagerEntry.GetAllServerInfo() + if len(serverInfos) == 0 { + return nil, true, errors.New("not found instance") + } + + serverNodes := make([]*infosync.ServerInfo, 0, len(serverInfos)) + for _, serverInfo := range serverInfos { + serverNodes = append(serverNodes, serverInfo) + } + return serverNodes, true, nil +} + +// GetPlanNotRetryableErrSchedulerExt returns mock scheduler.Extension which will generate non retryable error when planning. +func GetPlanNotRetryableErrSchedulerExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + return generateTaskExecutorNodes4Test() + }, + ).AnyTimes() + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(false).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + func(task *proto.Task) proto.Step { + return proto.StepDone + }, + ).AnyTimes() + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, _ *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + return nil, errors.New("not retryable err") + }, + ).AnyTimes() + + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + return mockScheduler +} + +// GetPlanErrSchedulerExt returns mock scheduler.Extension which will generate error when planning. +func GetPlanErrSchedulerExt(ctrl *gomock.Controller, testContext *TestContext) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + return generateTaskExecutorNodes4Test() + }, + ).AnyTimes() + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + func(task *proto.Task) proto.Step { + switch task.Step { + case proto.StepInit: + return proto.StepOne + case proto.StepOne: + return proto.StepTwo + default: + return proto.StepDone + } + }, + ).AnyTimes() + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + if task.Step == proto.StepInit { + if testContext.CallTime == 0 { + testContext.CallTime++ + return nil, errors.New("retryable err") + } + return [][]byte{ + []byte("task1"), + []byte("task2"), + []byte("task3"), + }, nil + } + if task.Step == proto.StepOne { + return [][]byte{ + []byte("task4"), + }, nil + } + return nil, nil + }, + ).AnyTimes() + + gomock.InOrder( + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(errors.New("not retryable err")), + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes(), + ) + return mockScheduler +} + +// GetMockRollbackSchedulerExt returns mock scheduler.Extension which will generate rollback subtasks. +func GetMockRollbackSchedulerExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + return generateTaskExecutorNodes4Test() + }, + ).AnyTimes() + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + func(task *proto.Task) proto.Step { + switch task.Step { + case proto.StepInit: + return proto.StepOne + default: + return proto.StepDone + } + }, + ).AnyTimes() + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + if task.Step == proto.StepInit { + return [][]byte{ + []byte("task1"), + []byte("task2"), + []byte("task3"), + }, nil + } + return nil, nil + }, + ).AnyTimes() + + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + return mockScheduler +} + +// GetMockDynamicDispatchExt returns mock scheduler.Extension which will generate subtask in multiple batches. +func GetMockDynamicDispatchExt(ctrl *gomock.Controller) scheduler.Extension { + mockScheduler := mockDispatch.NewMockExtension(ctrl) + mockScheduler.EXPECT().OnTick(gomock.Any(), gomock.Any()).Return().AnyTimes() + mockScheduler.EXPECT().GetEligibleInstances(gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ *proto.Task) ([]*infosync.ServerInfo, bool, error) { + return generateTaskExecutorNodes4Test() + }, + ).AnyTimes() + mockScheduler.EXPECT().IsRetryableErr(gomock.Any()).Return(true).AnyTimes() + mockScheduler.EXPECT().GetNextStep(gomock.Any()).DoAndReturn( + func(task *proto.Task) proto.Step { + switch task.Step { + case proto.StepInit: + return proto.StepOne + case proto.StepOne: + return proto.StepTwo + default: + return proto.StepDone + } + }, + ).AnyTimes() + mockScheduler.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( + func(_ context.Context, _ scheduler.TaskHandle, task *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + if task.Step == proto.StepInit { + return [][]byte{ + []byte("task"), + []byte("task"), + }, nil + } + + // step2 + if task.Step == proto.StepOne { + return [][]byte{ + []byte("task"), + }, nil + } + return nil, nil + }, + ).AnyTimes() + + mockScheduler.EXPECT().OnDone(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil).AnyTimes() + + return mockScheduler +} diff --git a/pkg/disttask/importinto/BUILD.bazel b/pkg/disttask/importinto/BUILD.bazel index ef229b5f347d8..978da54adb429 100644 --- a/pkg/disttask/importinto/BUILD.bazel +++ b/pkg/disttask/importinto/BUILD.bazel @@ -4,12 +4,12 @@ go_library( name = "importinto", srcs = [ "clean_s3.go", - "dispatcher.go", "encode_and_sort_operator.go", "job.go", "metrics.go", "planner.go", "proto.go", + "scheduler.go", "subtask_executor.go", "task_executor.go", "wrapper.go", @@ -31,10 +31,10 @@ go_library( "//br/pkg/storage", "//br/pkg/utils", "//pkg/config", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/handle", "//pkg/disttask/framework/planner", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/taskexecutor", "//pkg/disttask/framework/taskexecutor/execute", @@ -82,12 +82,12 @@ go_test( name = "importinto_test", timeout = "short", srcs = [ - "dispatcher_test.go", - "dispatcher_testkit_test.go", "encode_and_sort_operator_test.go", "job_testkit_test.go", "metrics_test.go", "planner_test.go", + "scheduler_test.go", + "scheduler_testkit_test.go", "subtask_executor_test.go", "task_executor_test.go", "task_executor_testkit_test.go", @@ -105,9 +105,9 @@ go_test( "//br/pkg/lightning/mydump", "//br/pkg/lightning/verification", "//pkg/ddl", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/planner", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/testutil", "//pkg/disttask/importinto/mock", diff --git a/pkg/disttask/importinto/clean_s3.go b/pkg/disttask/importinto/clean_s3.go index 5ae75137c46e7..e935163871222 100644 --- a/pkg/disttask/importinto/clean_s3.go +++ b/pkg/disttask/importinto/clean_s3.go @@ -21,19 +21,19 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/backend/external" "github.com/pingcap/tidb/br/pkg/lightning/log" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/util/logutil" "go.uber.org/zap" ) -var _ dispatcher.CleanUpRoutine = (*ImportCleanUpS3)(nil) +var _ scheduler.CleanUpRoutine = (*ImportCleanUpS3)(nil) -// ImportCleanUpS3 implements dispatcher.CleanUpRoutine. +// ImportCleanUpS3 implements scheduler.CleanUpRoutine. type ImportCleanUpS3 struct { } -func newImportCleanUpS3() dispatcher.CleanUpRoutine { +func newImportCleanUpS3() scheduler.CleanUpRoutine { return &ImportCleanUpS3{} } @@ -73,5 +73,5 @@ func (*ImportCleanUpS3) CleanUp(ctx context.Context, task *proto.Task) error { } func init() { - dispatcher.RegisterDispatcherCleanUpFactory(proto.ImportInto, newImportCleanUpS3) + scheduler.RegisterSchedulerCleanUpFactory(proto.ImportInto, newImportCleanUpS3) } diff --git a/pkg/disttask/importinto/metrics.go b/pkg/disttask/importinto/metrics.go index 2d69366181154..9c74a0e54de39 100644 --- a/pkg/disttask/importinto/metrics.go +++ b/pkg/disttask/importinto/metrics.go @@ -33,7 +33,7 @@ type taskMetrics struct { // taskMetricManager manages the metrics of IMPORT INTO tasks. // we have a set of metrics for each task, with different task_id const label. // metrics is passed by context value to avoid passing parameters everywhere. -// both dispatcher and taskExecutor might use it, to avoid registered again, +// both scheduler and taskExecutor might use it, to avoid registered again, // we add a manager to manage lifecycle of metrics for tasks. type taskMetricManager struct { sync.RWMutex diff --git a/pkg/disttask/importinto/proto.go b/pkg/disttask/importinto/proto.go index 158d507be9713..d26e08a430077 100644 --- a/pkg/disttask/importinto/proto.go +++ b/pkg/disttask/importinto/proto.go @@ -65,14 +65,14 @@ type TaskMeta struct { // running on the instance that initiate the IMPORT INTO. EligibleInstances []*infosync.ServerInfo // the file chunks to import, when import from server file, we need to pass those - // files to the framework dispatcher which might run on another instance. + // files to the framework scheduler which might run on another instance. // we use a map from engine ID to chunks since we need support split_file for CSV, - // so need to split them into engines before passing to dispatcher. + // so need to split them into engines before passing to scheduler. ChunkMap map[int32][]Chunk } // ImportStepMeta is the meta of import step. -// Dispatcher will split the task into subtasks(FileInfos -> Chunks) +// Scheduler will split the task into subtasks(FileInfos -> Chunks) // All the field should be serializable. type ImportStepMeta struct { // this is the engine ID, not the id in tidb_background_subtask table. diff --git a/pkg/disttask/importinto/dispatcher.go b/pkg/disttask/importinto/scheduler.go similarity index 76% rename from pkg/disttask/importinto/dispatcher.go rename to pkg/disttask/importinto/scheduler.go index 83ed6195a787a..b73cf888d7685 100644 --- a/pkg/disttask/importinto/dispatcher.go +++ b/pkg/disttask/importinto/scheduler.go @@ -30,10 +30,10 @@ import ( "github.com/pingcap/tidb/br/pkg/lightning/config" "github.com/pingcap/tidb/br/pkg/lightning/metric" "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/handle" "github.com/pingcap/tidb/pkg/disttask/framework/planner" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/errno" @@ -120,8 +120,8 @@ func (t *taskInfo) close(ctx context.Context) { } } -// ImportDispatcherExt is an extension of ImportDispatcher, exported for test. -type ImportDispatcherExt struct { +// ImportSchedulerExt is an extension of ImportScheduler, exported for test. +type ImportSchedulerExt struct { GlobalSort bool mu sync.RWMutex // NOTE: there's no need to sync for below 2 fields actually, since we add a restriction that only one @@ -138,37 +138,37 @@ type ImportDispatcherExt struct { disableTiKVImportMode atomic.Bool } -var _ dispatcher.Extension = (*ImportDispatcherExt)(nil) +var _ scheduler.Extension = (*ImportSchedulerExt)(nil) -// OnTick implements dispatcher.Extension interface. -func (dsp *ImportDispatcherExt) OnTick(ctx context.Context, task *proto.Task) { +// OnTick implements scheduler.Extension interface. +func (sch *ImportSchedulerExt) OnTick(ctx context.Context, task *proto.Task) { // only switch TiKV mode or register task when task is running if task.State != proto.TaskStateRunning { return } - dsp.switchTiKVMode(ctx, task) - dsp.registerTask(ctx, task) + sch.switchTiKVMode(ctx, task) + sch.registerTask(ctx, task) } -func (*ImportDispatcherExt) isImporting2TiKV(task *proto.Task) bool { +func (*ImportSchedulerExt) isImporting2TiKV(task *proto.Task) bool { return task.Step == StepImport || task.Step == StepWriteAndIngest } -func (dsp *ImportDispatcherExt) switchTiKVMode(ctx context.Context, task *proto.Task) { - dsp.updateCurrentTask(task) +func (sch *ImportSchedulerExt) switchTiKVMode(ctx context.Context, task *proto.Task) { + sch.updateCurrentTask(task) // only import step need to switch to IMPORT mode, // If TiKV is in IMPORT mode during checksum, coprocessor will time out. - if dsp.disableTiKVImportMode.Load() || !dsp.isImporting2TiKV(task) { + if sch.disableTiKVImportMode.Load() || !sch.isImporting2TiKV(task) { return } - if time.Since(dsp.lastSwitchTime.Load()) < config.DefaultSwitchTiKVModeInterval { + if time.Since(sch.lastSwitchTime.Load()) < config.DefaultSwitchTiKVModeInterval { return } - dsp.mu.Lock() - defer dsp.mu.Unlock() - if time.Since(dsp.lastSwitchTime.Load()) < config.DefaultSwitchTiKVModeInterval { + sch.mu.Lock() + defer sch.mu.Unlock() + if time.Since(sch.lastSwitchTime.Load()) < config.DefaultSwitchTiKVModeInterval { return } @@ -180,26 +180,26 @@ func (dsp *ImportDispatcherExt) switchTiKVMode(ctx context.Context, task *proto. } switcher.ToImportMode(ctx) pdCli.Close() - dsp.lastSwitchTime.Store(time.Now()) + sch.lastSwitchTime.Store(time.Now()) } -func (dsp *ImportDispatcherExt) registerTask(ctx context.Context, task *proto.Task) { - val, _ := dsp.taskInfoMap.LoadOrStore(task.ID, &taskInfo{taskID: task.ID}) +func (sch *ImportSchedulerExt) registerTask(ctx context.Context, task *proto.Task) { + val, _ := sch.taskInfoMap.LoadOrStore(task.ID, &taskInfo{taskID: task.ID}) info := val.(*taskInfo) info.register(ctx) } -func (dsp *ImportDispatcherExt) unregisterTask(ctx context.Context, task *proto.Task) { - if val, loaded := dsp.taskInfoMap.LoadAndDelete(task.ID); loaded { +func (sch *ImportSchedulerExt) unregisterTask(ctx context.Context, task *proto.Task) { + if val, loaded := sch.taskInfoMap.LoadAndDelete(task.ID); loaded { info := val.(*taskInfo) info.close(ctx) } } // OnNextSubtasksBatch generate batch of next stage's plan. -func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( +func (sch *ImportSchedulerExt) OnNextSubtasksBatch( ctx context.Context, - taskHandle dispatcher.TaskHandle, + taskHandle scheduler.TaskHandle, task *proto.Task, serverInfos []*infosync.ServerInfo, nextStep proto.Step, @@ -225,7 +225,7 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( metrics.BytesCounter.WithLabelValues(metric.StateTotalRestore).Add(float64(taskMeta.Plan.TotalFileSize)) } jobStep := importer.JobStepImporting - if dsp.GlobalSort { + if sch.GlobalSort { jobStep = importer.JobStepGlobalSorting } if err = startJob(ctx, logger, taskHandle, taskMeta, jobStep); err != nil { @@ -257,9 +257,9 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( return nil, err } case StepPostProcess: - dsp.switchTiKV2NormalMode(ctx, task, logger) + sch.switchTiKV2NormalMode(ctx, task, logger) failpoint.Inject("clearLastSwitchTime", func() { - dsp.lastSwitchTime.Store(time.Time{}) + sch.lastSwitchTime.Store(time.Time{}) }) if err = job2Step(ctx, logger, taskMeta, importer.JobStepValidating); err != nil { return nil, err @@ -268,10 +268,10 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( failpoint.Return(nil, errors.New("injected error after StepImport")) }) // we need get metas where checksum is stored. - if err := updateResult(taskHandle, task, taskMeta, dsp.GlobalSort); err != nil { + if err := updateResult(taskHandle, task, taskMeta, sch.GlobalSort); err != nil { return nil, err } - step := getStepOfEncode(dsp.GlobalSort) + step := getStepOfEncode(sch.GlobalSort) metas, err := taskHandle.GetPreviousSubtaskMetas(task.ID, step) if err != nil { return nil, err @@ -288,7 +288,7 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( Ctx: ctx, TaskID: task.ID, PreviousSubtaskMetas: previousSubtaskMetas, - GlobalSort: dsp.GlobalSort, + GlobalSort: sch.GlobalSort, NextTaskStep: nextStep, ExecuteNodesCnt: len(serverInfos), } @@ -308,8 +308,8 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( return metaBytes, nil } -// OnDone implements dispatcher.Extension interface. -func (dsp *ImportDispatcherExt) OnDone(ctx context.Context, handle dispatcher.TaskHandle, task *proto.Task) error { +// OnDone implements scheduler.Extension interface. +func (sch *ImportSchedulerExt) OnDone(ctx context.Context, handle scheduler.TaskHandle, task *proto.Task) error { logger := logutil.BgLogger().With( zap.Stringer("type", task.Type), zap.Int64("task-id", task.ID), @@ -322,16 +322,16 @@ func (dsp *ImportDispatcherExt) OnDone(ctx context.Context, handle dispatcher.Ta return errors.Trace(err) } if task.Error == nil { - return dsp.finishJob(ctx, logger, handle, task, taskMeta) + return sch.finishJob(ctx, logger, handle, task, taskMeta) } - if dispatcher.IsCancelledErr(task.Error) { - return dsp.cancelJob(ctx, handle, task, taskMeta, logger) + if scheduler.IsCancelledErr(task.Error) { + return sch.cancelJob(ctx, handle, task, taskMeta, logger) } - return dsp.failJob(ctx, handle, task, taskMeta, logger, task.Error.Error()) + return sch.failJob(ctx, handle, task, taskMeta, logger, task.Error.Error()) } -// GetEligibleInstances implements dispatcher.Extension interface. -func (*ImportDispatcherExt) GetEligibleInstances(ctx context.Context, task *proto.Task) ([]*infosync.ServerInfo, bool, error) { +// GetEligibleInstances implements scheduler.Extension interface. +func (*ImportSchedulerExt) GetEligibleInstances(ctx context.Context, task *proto.Task) ([]*infosync.ServerInfo, bool, error) { taskMeta := &TaskMeta{} err := json.Unmarshal(task.Meta, taskMeta) if err != nil { @@ -340,21 +340,21 @@ func (*ImportDispatcherExt) GetEligibleInstances(ctx context.Context, task *prot if len(taskMeta.EligibleInstances) > 0 { return taskMeta.EligibleInstances, false, nil } - serverInfo, err := dispatcher.GenerateTaskExecutorNodes(ctx) + serverInfo, err := scheduler.GenerateTaskExecutorNodes(ctx) return serverInfo, true, err } -// IsRetryableErr implements dispatcher.Extension interface. -func (*ImportDispatcherExt) IsRetryableErr(error) bool { +// IsRetryableErr implements scheduler.Extension interface. +func (*ImportSchedulerExt) IsRetryableErr(error) bool { // TODO: check whether the error is retryable. return false } -// GetNextStep implements dispatcher.Extension interface. -func (dsp *ImportDispatcherExt) GetNextStep(task *proto.Task) proto.Step { +// GetNextStep implements scheduler.Extension interface. +func (sch *ImportSchedulerExt) GetNextStep(task *proto.Task) proto.Step { switch task.Step { case proto.StepInit: - if dsp.GlobalSort { + if sch.GlobalSort { return StepEncodeAndSort } return StepImport @@ -370,14 +370,14 @@ func (dsp *ImportDispatcherExt) GetNextStep(task *proto.Task) proto.Step { } } -func (dsp *ImportDispatcherExt) switchTiKV2NormalMode(ctx context.Context, task *proto.Task, logger *zap.Logger) { - dsp.updateCurrentTask(task) - if dsp.disableTiKVImportMode.Load() { +func (sch *ImportSchedulerExt) switchTiKV2NormalMode(ctx context.Context, task *proto.Task, logger *zap.Logger) { + sch.updateCurrentTask(task) + if sch.disableTiKVImportMode.Load() { return } - dsp.mu.Lock() - defer dsp.mu.Unlock() + sch.mu.Lock() + defer sch.mu.Unlock() pdCli, switcher, err := importer.GetTiKVModeSwitcherWithPDClient(ctx, logger) if err != nil { @@ -388,58 +388,58 @@ func (dsp *ImportDispatcherExt) switchTiKV2NormalMode(ctx context.Context, task pdCli.Close() // clear it, so next task can switch TiKV mode again. - dsp.lastSwitchTime.Store(time.Time{}) + sch.lastSwitchTime.Store(time.Time{}) } -func (dsp *ImportDispatcherExt) updateCurrentTask(task *proto.Task) { - if dsp.currTaskID.Swap(task.ID) != task.ID { +func (sch *ImportSchedulerExt) updateCurrentTask(task *proto.Task) { + if sch.currTaskID.Swap(task.ID) != task.ID { taskMeta := &TaskMeta{} if err := json.Unmarshal(task.Meta, taskMeta); err == nil { // for raftkv2, switch mode in local backend - dsp.disableTiKVImportMode.Store(taskMeta.Plan.DisableTiKVImportMode || taskMeta.Plan.IsRaftKV2) + sch.disableTiKVImportMode.Store(taskMeta.Plan.DisableTiKVImportMode || taskMeta.Plan.IsRaftKV2) } } } -type importDispatcher struct { - *dispatcher.BaseDispatcher +type importScheduler struct { + *scheduler.BaseScheduler } -func newImportDispatcher(ctx context.Context, taskMgr dispatcher.TaskManager, - serverID string, task *proto.Task) dispatcher.Dispatcher { +func newImportScheduler(ctx context.Context, taskMgr scheduler.TaskManager, + serverID string, task *proto.Task) scheduler.Scheduler { metrics := metricsManager.getOrCreateMetrics(task.ID) subCtx := metric.WithCommonMetric(ctx, metrics) - dsp := importDispatcher{ - BaseDispatcher: dispatcher.NewBaseDispatcher(subCtx, taskMgr, serverID, task), + sch := importScheduler{ + BaseScheduler: scheduler.NewBaseScheduler(subCtx, taskMgr, serverID, task), } - return &dsp + return &sch } -func (dsp *importDispatcher) Init() (err error) { +func (sch *importScheduler) Init() (err error) { defer func() { if err != nil { // if init failed, close is not called, so we need to unregister here. - metricsManager.unregister(dsp.Task.ID) + metricsManager.unregister(sch.Task.ID) } }() taskMeta := &TaskMeta{} - if err = json.Unmarshal(dsp.BaseDispatcher.Task.Meta, taskMeta); err != nil { + if err = json.Unmarshal(sch.BaseScheduler.Task.Meta, taskMeta); err != nil { return errors.Annotate(err, "unmarshal task meta failed") } - dsp.BaseDispatcher.Extension = &ImportDispatcherExt{ + sch.BaseScheduler.Extension = &ImportSchedulerExt{ GlobalSort: taskMeta.Plan.CloudStorageURI != "", } - return dsp.BaseDispatcher.Init() + return sch.BaseScheduler.Init() } -func (dsp *importDispatcher) Close() { - metricsManager.unregister(dsp.Task.ID) - dsp.BaseDispatcher.Close() +func (sch *importScheduler) Close() { + metricsManager.unregister(sch.Task.ID) + sch.BaseScheduler.Close() } // nolint:deadcode -func dropTableIndexes(ctx context.Context, handle dispatcher.TaskHandle, taskMeta *TaskMeta, logger *zap.Logger) error { +func dropTableIndexes(ctx context.Context, handle scheduler.TaskHandle, taskMeta *TaskMeta, logger *zap.Logger) error { tblInfo := taskMeta.Plan.TableInfo tableName := common.UniqueTable(taskMeta.Plan.DBName, tblInfo.Name.L) @@ -536,7 +536,7 @@ func getStepOfEncode(globalSort bool) proto.Step { } // we will update taskMeta in place and make task.Meta point to the new taskMeta. -func updateResult(handle dispatcher.TaskHandle, task *proto.Task, taskMeta *TaskMeta, globalSort bool) error { +func updateResult(handle scheduler.TaskHandle, task *proto.Task, taskMeta *TaskMeta, globalSort bool) error { stepOfEncode := getStepOfEncode(globalSort) metas, err := handle.GetPreviousSubtaskMetas(task.ID, stepOfEncode) if err != nil { @@ -570,7 +570,7 @@ func updateResult(handle dispatcher.TaskHandle, task *proto.Task, taskMeta *Task return updateMeta(task, taskMeta) } -func getLoadedRowCountOnGlobalSort(handle dispatcher.TaskHandle, task *proto.Task) (uint64, error) { +func getLoadedRowCountOnGlobalSort(handle scheduler.TaskHandle, task *proto.Task) (uint64, error) { metas, err := handle.GetPreviousSubtaskMetas(task.ID, StepWriteAndIngest) if err != nil { return 0, err @@ -587,7 +587,7 @@ func getLoadedRowCountOnGlobalSort(handle dispatcher.TaskHandle, task *proto.Tas return loadedRowCount, nil } -func startJob(ctx context.Context, logger *zap.Logger, taskHandle dispatcher.TaskHandle, taskMeta *TaskMeta, jobStep string) error { +func startJob(ctx context.Context, logger *zap.Logger, taskHandle scheduler.TaskHandle, taskMeta *TaskMeta, jobStep string) error { failpoint.Inject("syncBeforeJobStarted", func() { TestSyncChan <- struct{}{} <-TestSyncChan @@ -596,8 +596,8 @@ func startJob(ctx context.Context, logger *zap.Logger, taskHandle dispatcher.Tas // we consider all errors as retryable errors, except context done. // the errors include errors happened when communicate with PD and TiKV. // we didn't consider system corrupt cases like system table dropped/altered. - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - err := handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + err := handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, taskHandle.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) @@ -616,11 +616,11 @@ func job2Step(ctx context.Context, logger *zap.Logger, taskMeta *TaskMeta, step if err != nil { return err } - // todo: use dispatcher.TaskHandle - // we might call this in taskExecutor later, there's no dispatcher.Extension, so we use taskManager here. + // todo: use scheduler.TaskHandle + // we might call this in taskExecutor later, there's no scheduler.Extension, so we use taskManager here. // retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - return handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + return handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, taskManager.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) @@ -630,14 +630,14 @@ func job2Step(ctx context.Context, logger *zap.Logger, taskMeta *TaskMeta, step ) } -func (dsp *ImportDispatcherExt) finishJob(ctx context.Context, logger *zap.Logger, - taskHandle dispatcher.TaskHandle, task *proto.Task, taskMeta *TaskMeta) error { +func (sch *ImportSchedulerExt) finishJob(ctx context.Context, logger *zap.Logger, + taskHandle scheduler.TaskHandle, task *proto.Task, taskMeta *TaskMeta) error { // we have already switch import-mode when switch to post-process step. - dsp.unregisterTask(ctx, task) + sch.unregisterTask(ctx, task) summary := &importer.JobSummary{ImportedRows: taskMeta.Result.LoadedRowCnt} // retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - return handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + return handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, taskHandle.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) @@ -647,13 +647,13 @@ func (dsp *ImportDispatcherExt) finishJob(ctx context.Context, logger *zap.Logge ) } -func (dsp *ImportDispatcherExt) failJob(ctx context.Context, taskHandle dispatcher.TaskHandle, task *proto.Task, +func (sch *ImportSchedulerExt) failJob(ctx context.Context, taskHandle scheduler.TaskHandle, task *proto.Task, taskMeta *TaskMeta, logger *zap.Logger, errorMsg string) error { - dsp.switchTiKV2NormalMode(ctx, task, logger) - dsp.unregisterTask(ctx, task) + sch.switchTiKV2NormalMode(ctx, task, logger) + sch.unregisterTask(ctx, task) // retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - return handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + return handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, taskHandle.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) @@ -663,13 +663,13 @@ func (dsp *ImportDispatcherExt) failJob(ctx context.Context, taskHandle dispatch ) } -func (dsp *ImportDispatcherExt) cancelJob(ctx context.Context, taskHandle dispatcher.TaskHandle, task *proto.Task, +func (sch *ImportSchedulerExt) cancelJob(ctx context.Context, taskHandle scheduler.TaskHandle, task *proto.Task, meta *TaskMeta, logger *zap.Logger) error { - dsp.switchTiKV2NormalMode(ctx, task, logger) - dsp.unregisterTask(ctx, task) + sch.switchTiKV2NormalMode(ctx, task, logger) + sch.unregisterTask(ctx, task) // retry for 3+6+12+24+(30-4)*30 ~= 825s ~= 14 minutes - backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) - return handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, + backoffer := backoff.NewExponential(scheduler.RetrySQLInterval, 2, scheduler.RetrySQLMaxInterval) + return handle.RunWithRetry(ctx, scheduler.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { return true, taskHandle.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) @@ -713,5 +713,5 @@ func stepStr(step proto.Step) string { } func init() { - dispatcher.RegisterDispatcherFactory(proto.ImportInto, newImportDispatcher) + scheduler.RegisterSchedulerFactory(proto.ImportInto, newImportScheduler) } diff --git a/pkg/disttask/importinto/dispatcher_test.go b/pkg/disttask/importinto/scheduler_test.go similarity index 78% rename from pkg/disttask/importinto/dispatcher_test.go rename to pkg/disttask/importinto/scheduler_test.go index 839dd14cf8e7d..53dbf593102d4 100644 --- a/pkg/disttask/importinto/dispatcher_test.go +++ b/pkg/disttask/importinto/scheduler_test.go @@ -21,8 +21,8 @@ import ( "testing" "github.com/pingcap/failpoint" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/domain/infosync" "github.com/pingcap/tidb/pkg/executor/importer" "github.com/stretchr/testify/require" @@ -44,7 +44,7 @@ func (s *importIntoSuite) enableFailPoint(path, term string) { }) } -func (s *importIntoSuite) TestDispatcherGetEligibleInstances() { +func (s *importIntoSuite) TestSchedulerGetEligibleInstances() { makeFailpointRes := func(v interface{}) string { bytes, err := json.Marshal(v) s.NoError(err) @@ -61,11 +61,11 @@ func (s *importIntoSuite) TestDispatcherGetEligibleInstances() { } mockedAllServerInfos := makeFailpointRes(serverInfoMap) - dsp := ImportDispatcherExt{} + sch := ImportSchedulerExt{} task := &proto.Task{Meta: []byte("{}")} ctx := context.WithValue(context.Background(), "etcd", true) s.enableFailPoint("github.com/pingcap/tidb/pkg/domain/infosync/mockGetAllServerInfo", mockedAllServerInfos) - eligibleInstances, _, err := dsp.GetEligibleInstances(ctx, task) + eligibleInstances, _, err := sch.GetEligibleInstances(ctx, task) s.NoError(err) // order of slice is not stable, change to map resultMap := map[string]*infosync.ServerInfo{} @@ -75,7 +75,7 @@ func (s *importIntoSuite) TestDispatcherGetEligibleInstances() { s.Equal(serverInfoMap, resultMap) task.Meta = []byte(`{"EligibleInstances":[{"ip": "1.1.1.1", "listening_port": 4000}]}`) - eligibleInstances, _, err = dsp.GetEligibleInstances(ctx, task) + eligibleInstances, _, err = sch.GetEligibleInstances(ctx, task) s.NoError(err) s.Equal([]*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}}, eligibleInstances) } @@ -89,26 +89,26 @@ func (s *importIntoSuite) TestUpdateCurrentTask() { bs, err := json.Marshal(taskMeta) require.NoError(s.T(), err) - dsp := ImportDispatcherExt{} - require.Equal(s.T(), int64(0), dsp.currTaskID.Load()) - require.False(s.T(), dsp.disableTiKVImportMode.Load()) + sch := ImportSchedulerExt{} + require.Equal(s.T(), int64(0), sch.currTaskID.Load()) + require.False(s.T(), sch.disableTiKVImportMode.Load()) - dsp.updateCurrentTask(&proto.Task{ + sch.updateCurrentTask(&proto.Task{ ID: 1, Meta: bs, }) - require.Equal(s.T(), int64(1), dsp.currTaskID.Load()) - require.True(s.T(), dsp.disableTiKVImportMode.Load()) + require.Equal(s.T(), int64(1), sch.currTaskID.Load()) + require.True(s.T(), sch.disableTiKVImportMode.Load()) - dsp.updateCurrentTask(&proto.Task{ + sch.updateCurrentTask(&proto.Task{ ID: 1, Meta: bs, }) - require.Equal(s.T(), int64(1), dsp.currTaskID.Load()) - require.True(s.T(), dsp.disableTiKVImportMode.Load()) + require.Equal(s.T(), int64(1), sch.currTaskID.Load()) + require.True(s.T(), sch.disableTiKVImportMode.Load()) } -func (s *importIntoSuite) TestDispatcherInit() { +func (s *importIntoSuite) TestSchedulerInit() { meta := TaskMeta{ Plan: importer.Plan{ CloudStorageURI: "", @@ -116,42 +116,42 @@ func (s *importIntoSuite) TestDispatcherInit() { } bytes, err := json.Marshal(meta) s.NoError(err) - dsp := importDispatcher{ - BaseDispatcher: &dispatcher.BaseDispatcher{ + sch := importScheduler{ + BaseScheduler: &scheduler.BaseScheduler{ Task: &proto.Task{ Meta: bytes, }, }, } - s.NoError(dsp.Init()) - s.False(dsp.Extension.(*ImportDispatcherExt).GlobalSort) + s.NoError(sch.Init()) + s.False(sch.Extension.(*ImportSchedulerExt).GlobalSort) meta.Plan.CloudStorageURI = "s3://test" bytes, err = json.Marshal(meta) s.NoError(err) - dsp = importDispatcher{ - BaseDispatcher: &dispatcher.BaseDispatcher{ + sch = importScheduler{ + BaseScheduler: &scheduler.BaseScheduler{ Task: &proto.Task{ Meta: bytes, }, }, } - s.NoError(dsp.Init()) - s.True(dsp.Extension.(*ImportDispatcherExt).GlobalSort) + s.NoError(sch.Init()) + s.True(sch.Extension.(*ImportSchedulerExt).GlobalSort) } func (s *importIntoSuite) TestGetNextStep() { task := &proto.Task{ Step: proto.StepInit, } - ext := &ImportDispatcherExt{} + ext := &ImportSchedulerExt{} for _, nextStep := range []proto.Step{StepImport, StepPostProcess, proto.StepDone} { s.Equal(nextStep, ext.GetNextStep(task)) task.Step = nextStep } task.Step = proto.StepInit - ext = &ImportDispatcherExt{GlobalSort: true} + ext = &ImportSchedulerExt{GlobalSort: true} for _, nextStep := range []proto.Step{StepEncodeAndSort, StepMergeSort, StepWriteAndIngest, StepPostProcess, proto.StepDone} { s.Equal(nextStep, ext.GetNextStep(task)) @@ -176,7 +176,7 @@ func (s *importIntoSuite) TestGetStepOfEncode() { } func TestIsImporting2TiKV(t *testing.T) { - ext := &ImportDispatcherExt{} + ext := &ImportSchedulerExt{} require.False(t, ext.isImporting2TiKV(&proto.Task{Step: StepEncodeAndSort})) require.False(t, ext.isImporting2TiKV(&proto.Task{Step: StepMergeSort})) require.False(t, ext.isImporting2TiKV(&proto.Task{Step: StepPostProcess})) diff --git a/pkg/disttask/importinto/dispatcher_testkit_test.go b/pkg/disttask/importinto/scheduler_testkit_test.go similarity index 94% rename from pkg/disttask/importinto/dispatcher_testkit_test.go rename to pkg/disttask/importinto/scheduler_testkit_test.go index 6e2e6ee2225e0..b41a7e46c1d69 100644 --- a/pkg/disttask/importinto/dispatcher_testkit_test.go +++ b/pkg/disttask/importinto/scheduler_testkit_test.go @@ -24,8 +24,8 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/br/pkg/lightning/backend/external" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/importinto" "github.com/pingcap/tidb/pkg/domain/infosync" @@ -37,7 +37,7 @@ import ( "github.com/tikv/client-go/v2/util" ) -func TestDispatcherExtLocalSort(t *testing.T) { +func TestSchedulerExtLocalSort(t *testing.T) { store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) pool := pools.NewResourcePool(func() (pools.Resource, error) { @@ -48,7 +48,7 @@ func TestDispatcherExtLocalSort(t *testing.T) { ctx = util.WithInternalSourceType(ctx, "taskManager") mgr := storage.NewTaskManager(pool) storage.SetTaskManager(mgr) - dsp, err := dispatcher.NewManager(util.WithInternalSourceType(ctx, "dispatcher"), mgr, "host:port") + sch, err := scheduler.NewManager(util.WithInternalSourceType(ctx, "scheduler"), mgr, "host:port") require.NoError(t, err) // create job @@ -90,8 +90,8 @@ func TestDispatcherExtLocalSort(t *testing.T) { task.ID = taskID // to import stage, job should be running - d := dsp.MockDispatcher(task) - ext := importinto.ImportDispatcherExt{} + d := sch.MockScheduler(task) + ext := importinto.ImportSchedulerExt{} serverInfos, _, err := ext.GetEligibleInstances(context.Background(), task) require.NoError(t, err) subtaskMetas, err := ext.OnNextSubtasksBatch(ctx, d, task, serverInfos, ext.GetNextStep(task)) @@ -168,12 +168,12 @@ func TestDispatcherExtLocalSort(t *testing.T) { require.Equal(t, "cancelled", gotJobInfo.Status) } -func TestDispatcherExtGlobalSort(t *testing.T) { - // Domain start dispatcher manager automatically, we need to disable it as +func TestSchedulerExtGlobalSort(t *testing.T) { + // Domain start scheduler manager automatically, we need to disable it as // we test import task management in this case. - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/disableDispatcherManager", "return(true)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/disableSchedulerManager", "return(true)")) t.Cleanup(func() { - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/disableDispatcherManager")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/disableSchedulerManager")) }) store := testkit.CreateMockStore(t) tk := testkit.NewTestKit(t, store) @@ -185,7 +185,7 @@ func TestDispatcherExtGlobalSort(t *testing.T) { ctx = util.WithInternalSourceType(ctx, "taskManager") mgr := storage.NewTaskManager(pool) storage.SetTaskManager(mgr) - dsp, err := dispatcher.NewManager(util.WithInternalSourceType(ctx, "dispatcher"), mgr, "host:port") + sch, err := scheduler.NewManager(util.WithInternalSourceType(ctx, "scheduler"), mgr, "host:port") require.NoError(t, err) // create job @@ -235,8 +235,8 @@ func TestDispatcherExtGlobalSort(t *testing.T) { task.ID = taskID // to encode-sort stage, job should be running - d := dsp.MockDispatcher(task) - ext := importinto.ImportDispatcherExt{ + d := sch.MockScheduler(task) + ext := importinto.ImportSchedulerExt{ GlobalSort: true, } serverInfos, _, err := ext.GetEligibleInstances(context.Background(), task) diff --git a/pkg/disttask/importinto/task_executor.go b/pkg/disttask/importinto/task_executor.go index de419a8fe540f..73c0fc8ab311e 100644 --- a/pkg/disttask/importinto/task_executor.go +++ b/pkg/disttask/importinto/task_executor.go @@ -133,7 +133,7 @@ func (s *importStepExecutor) RunSubtask(ctx context.Context, subtask *proto.Subt // Unlike in Lightning, we start an index engine for each subtask, // whereas previously there was only a single index engine globally. // This is because the executor currently does not have a post-processing mechanism. - // If we import the index in `cleanupSubtaskEnv`, the dispatcher will not wait for the import to complete. + // If we import the index in `cleanupSubtaskEnv`, the scheduler will not wait for the import to complete. // Multiple index engines may suffer performance degradation due to range overlap. // These issues will be alleviated after we integrate s3 sorter. // engineID = -1, -2, -3, ... diff --git a/pkg/domain/BUILD.bazel b/pkg/domain/BUILD.bazel index 969e0d875c452..2f1a8c876d524 100644 --- a/pkg/domain/BUILD.bazel +++ b/pkg/domain/BUILD.bazel @@ -29,7 +29,7 @@ go_library( "//pkg/ddl/placement", "//pkg/ddl/schematracker", "//pkg/ddl/util", - "//pkg/disttask/framework/dispatcher", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/taskexecutor", "//pkg/domain/globalconfigsync", diff --git a/pkg/domain/domain.go b/pkg/domain/domain.go index 03b3afbca3b2b..0499e55255180 100644 --- a/pkg/domain/domain.go +++ b/pkg/domain/domain.go @@ -40,7 +40,7 @@ import ( "github.com/pingcap/tidb/pkg/ddl/placement" "github.com/pingcap/tidb/pkg/ddl/schematracker" ddlutil "github.com/pingcap/tidb/pkg/ddl/util" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor" "github.com/pingcap/tidb/pkg/domain/globalconfigsync" @@ -1506,24 +1506,24 @@ func (do *Domain) distTaskFrameworkLoop(ctx context.Context, taskManager *storag logutil.BgLogger().Info("dist task executor manager stopped") }() - var dispatcherManager *dispatcher.Manager + var schedulerManager *scheduler.Manager startDispatchIfNeeded := func() { - if dispatcherManager != nil && dispatcherManager.Initialized() { + if schedulerManager != nil && schedulerManager.Initialized() { return } var err error - dispatcherManager, err = dispatcher.NewManager(ctx, taskManager, serverID) + schedulerManager, err = scheduler.NewManager(ctx, taskManager, serverID) if err != nil { - logutil.BgLogger().Error("failed to create a dist task dispatcher manager", zap.Error(err)) + logutil.BgLogger().Error("failed to create a dist task scheduler manager", zap.Error(err)) return } - dispatcherManager.Start() + schedulerManager.Start() } stopDispatchIfNeeded := func() { - if dispatcherManager != nil && dispatcherManager.Initialized() { - logutil.BgLogger().Info("stopping dist task dispatcher manager because the current node is not DDL owner anymore", zap.String("id", do.ddl.GetID())) - dispatcherManager.Stop() - logutil.BgLogger().Info("dist task dispatcher manager stopped", zap.String("id", do.ddl.GetID())) + if schedulerManager != nil && schedulerManager.Initialized() { + logutil.BgLogger().Info("stopping dist task scheduler manager because the current node is not DDL owner anymore", zap.String("id", do.ddl.GetID())) + schedulerManager.Stop() + logutil.BgLogger().Info("dist task scheduler manager stopped", zap.String("id", do.ddl.GetID())) } } diff --git a/pkg/metrics/disttask.go b/pkg/metrics/disttask.go index 97f45322a9d29..dc8680d84846e 100644 --- a/pkg/metrics/disttask.go +++ b/pkg/metrics/disttask.go @@ -32,10 +32,10 @@ const ( // status for task const ( - DispatchingStatus = "dispatching" - WaitingStatus = "waiting" - RunningStatus = "running" - CompletedStatus = "completed" + SchedulingStatus = "scheduling" + WaitingStatus = "waiting" + RunningStatus = "running" + CompletedStatus = "completed" ) var ( @@ -132,13 +132,13 @@ func UpdateMetricsForAddTask(task *proto.Task) { func UpdateMetricsForDispatchTask(id int64, taskType proto.TaskType) { DistTaskGauge.WithLabelValues(taskType.String(), WaitingStatus).Dec() DistTaskStarttimeGauge.DeleteLabelValues(taskType.String(), WaitingStatus, fmt.Sprint(id)) - DistTaskStarttimeGauge.WithLabelValues(taskType.String(), DispatchingStatus, fmt.Sprint(id)).SetToCurrentTime() + DistTaskStarttimeGauge.WithLabelValues(taskType.String(), SchedulingStatus, fmt.Sprint(id)).SetToCurrentTime() } // UpdateMetricsForRunTask update metrics when a task starts running func UpdateMetricsForRunTask(task *proto.Task) { - DistTaskStarttimeGauge.DeleteLabelValues(task.Type.String(), DispatchingStatus, fmt.Sprint(task.ID)) - DistTaskGauge.WithLabelValues(task.Type.String(), DispatchingStatus).Dec() + DistTaskStarttimeGauge.DeleteLabelValues(task.Type.String(), SchedulingStatus, fmt.Sprint(task.ID)) + DistTaskGauge.WithLabelValues(task.Type.String(), SchedulingStatus).Dec() DistTaskGauge.WithLabelValues(task.Type.String(), RunningStatus).Inc() } diff --git a/tests/realtikvtest/addindextest1/BUILD.bazel b/tests/realtikvtest/addindextest1/BUILD.bazel index 9126f15f9bca9..69a797e2d5316 100644 --- a/tests/realtikvtest/addindextest1/BUILD.bazel +++ b/tests/realtikvtest/addindextest1/BUILD.bazel @@ -12,8 +12,8 @@ go_test( "//pkg/config", "//pkg/ddl", "//pkg/ddl/util/callback", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/errno", "//pkg/kv", "//pkg/parser/model", diff --git a/tests/realtikvtest/addindextest1/disttask_test.go b/tests/realtikvtest/addindextest1/disttask_test.go index a97f032559ea3..23397203b6a69 100644 --- a/tests/realtikvtest/addindextest1/disttask_test.go +++ b/tests/realtikvtest/addindextest1/disttask_test.go @@ -21,8 +21,8 @@ import ( "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl" "github.com/pingcap/tidb/pkg/ddl/util/callback" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/errno" "github.com/pingcap/tidb/pkg/kv" "github.com/pingcap/tidb/pkg/parser/model" @@ -154,7 +154,7 @@ func TestAddIndexDistPauseAndResume(t *testing.T) { <-ddl.TestSyncChan } - dispatcher.MockDMLExecutionOnPausedState = func(task *proto.Task) { + scheduler.MockDMLExecutionOnPausedState = func(task *proto.Task) { row := tk1.MustQuery("select job_id from mysql.tidb_ddl_job").Rows() require.Equal(t, 1, len(row)) jobID := row[0][0].(string) @@ -169,13 +169,13 @@ func TestAddIndexDistPauseAndResume(t *testing.T) { } require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/mockDMLExecutionAddIndexSubTaskFinish", "3*return(true)")) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDMLExecutionOnPausedState", "return(true)")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDMLExecutionOnPausedState", "return(true)")) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/syncDDLTaskPause", "return()")) tk.MustExec(`set global tidb_enable_dist_task=1;`) tk.MustExec("alter table t add index idx1(a);") tk.MustExec("admin check table t;") require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/mockDMLExecutionAddIndexSubTaskFinish")) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/mockDMLExecutionOnPausedState")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockDMLExecutionOnPausedState")) // dist task succeed, job paused and resumed. var hook = &callback.TestDDLCallback{Do: dom} diff --git a/tests/realtikvtest/addindextest2/BUILD.bazel b/tests/realtikvtest/addindextest2/BUILD.bazel index 9caa91fe19679..6c60e45eac88f 100644 --- a/tests/realtikvtest/addindextest2/BUILD.bazel +++ b/tests/realtikvtest/addindextest2/BUILD.bazel @@ -13,7 +13,7 @@ go_test( "//br/pkg/storage", "//pkg/config", "//pkg/ddl/util/callback", - "//pkg/disttask/framework/dispatcher", + "//pkg/disttask/framework/scheduler", "//pkg/parser/model", "//pkg/sessionctx/variable", "//pkg/testkit", diff --git a/tests/realtikvtest/addindextest2/global_sort_test.go b/tests/realtikvtest/addindextest2/global_sort_test.go index 2dc503ab7eeb7..63f25cfd8d44a 100644 --- a/tests/realtikvtest/addindextest2/global_sort_test.go +++ b/tests/realtikvtest/addindextest2/global_sort_test.go @@ -28,7 +28,7 @@ import ( "github.com/pingcap/tidb/br/pkg/storage" "github.com/pingcap/tidb/pkg/config" "github.com/pingcap/tidb/pkg/ddl/util/callback" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/parser/model" "github.com/pingcap/tidb/pkg/sessionctx/variable" "github.com/pingcap/tidb/pkg/testkit" @@ -81,7 +81,7 @@ func TestGlobalSortBasic(t *testing.T) { store, dom := realtikvtest.CreateMockStoreAndDomainAndSetup(t) tk := testkit.NewTestKit(t, store) - require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/WaitCleanUpFinished", "return()")) + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished", "return()")) tk.MustExec("drop database if exists addindexlit;") tk.MustExec("create database addindexlit;") tk.MustExec("use addindexlit;") @@ -118,17 +118,17 @@ func TestGlobalSortBasic(t *testing.T) { tk.MustExec("alter table t add index idx(a);") dom.DDL().SetHook(origin) tk.MustExec("admin check table t;") - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished checkFileCleaned(t, jobID, cloudStorageURI) require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/forceMergeSort", "return()")) tk.MustExec("alter table t add index idx1(a);") dom.DDL().SetHook(origin) tk.MustExec("admin check table t;") - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished checkFileCleaned(t, jobID, cloudStorageURI) - require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/WaitCleanUpFinished")) + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished")) require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/forceMergeSort")) } diff --git a/tests/realtikvtest/importintotest/BUILD.bazel b/tests/realtikvtest/importintotest/BUILD.bazel index 19952965a5a00..00bdb06074ffb 100644 --- a/tests/realtikvtest/importintotest/BUILD.bazel +++ b/tests/realtikvtest/importintotest/BUILD.bazel @@ -23,8 +23,8 @@ go_test( "//br/pkg/streamhelper", "//br/pkg/utils", "//pkg/config", - "//pkg/disttask/framework/dispatcher", "//pkg/disttask/framework/proto", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/framework/taskexecutor", "//pkg/disttask/importinto", diff --git a/tests/realtikvtest/importintotest/import_into_test.go b/tests/realtikvtest/importintotest/import_into_test.go index 17b0f0aa00a24..9b7b84792b35e 100644 --- a/tests/realtikvtest/importintotest/import_into_test.go +++ b/tests/realtikvtest/importintotest/import_into_test.go @@ -35,8 +35,8 @@ import ( "github.com/pingcap/tidb/br/pkg/mock" "github.com/pingcap/tidb/br/pkg/mock/mocklocal" "github.com/pingcap/tidb/br/pkg/utils" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" "github.com/pingcap/tidb/pkg/disttask/framework/proto" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/importinto" "github.com/pingcap/tidb/pkg/domain/infosync" @@ -829,7 +829,7 @@ func (s *mockGCSSuite) TestImportMode() { // NOTE: this case only runs when current instance is TiDB owner, if you run it locally, // better start a cluster without TiDB instance. s.enableFailpoint("github.com/pingcap/tidb/pkg/parser/ast/forceRedactURL", "return(true)") - s.enableFailpoint("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/WaitCleanUpFinished", "return()") + s.enableFailpoint("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished", "return()") sql := fmt.Sprintf(`IMPORT INTO load_data.import_mode FROM 'gs://test-load/import_mode-*.tsv?access-key=aaaaaa&secret-access-key=bbbbbb&endpoint=%s'`, gcsEndpoint) rows := s.tk.MustQuery(sql).Rows() s.Len(rows, 1) @@ -837,7 +837,7 @@ func (s *mockGCSSuite) TestImportMode() { s.NoError(err) s.tk.MustQuery("SELECT * FROM load_data.import_mode;").Sort().Check(testkit.Rows("1 11 111")) s.Greater(intoNormalTime, intoImportTime) - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished s.checkTaskMetaRedacted(int64(jobID)) // after import step, we should enter normal mode, i.e. we only call ToImportMode once intoNormalTime, intoImportTime = time.Time{}, time.Time{} @@ -853,7 +853,7 @@ func (s *mockGCSSuite) TestImportMode() { s.Greater(intoNormalTime, intoImportTime) s.NoError(failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/importinto/clearLastSwitchTime")) s.NoError(failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/importinto/waitBeforePostProcess")) - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished // test disable_tikv_import_mode, should not call ToImportMode and ToNormalMode s.tk.MustExec("truncate table load_data.import_mode;") @@ -861,7 +861,7 @@ func (s *mockGCSSuite) TestImportMode() { s.tk.MustQuery(sql) s.tk.MustQuery("SELECT * FROM load_data.import_mode;").Sort().Check(testkit.Rows("1 11 111")) s.tk.MustExec("truncate table load_data.import_mode;") - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished // test with multirocksdb s.enableFailpoint("github.com/pingcap/tidb/pkg/ddl/util/IsRaftKv2", "return(true)") @@ -870,7 +870,7 @@ func (s *mockGCSSuite) TestImportMode() { s.tk.MustQuery(sql) s.tk.MustQuery("SELECT * FROM load_data.import_mode;").Sort().Check(testkit.Rows("1 11 111")) s.tk.MustExec("truncate table load_data.import_mode;") - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished s.NoError(failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/util/IsRaftKv2")) @@ -886,9 +886,9 @@ func (s *mockGCSSuite) TestImportMode() { err = s.tk.QueryToErr(sql) s.Error(err) s.Greater(intoNormalTime, intoImportTime) - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished s.checkTaskMetaRedacted(importer.TestLastImportJobID.Load()) - s.NoError(failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/WaitCleanUpFinished")) + s.NoError(failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished")) } func (s *mockGCSSuite) TestRegisterTask() { diff --git a/tests/realtikvtest/importintotest4/BUILD.bazel b/tests/realtikvtest/importintotest4/BUILD.bazel index ffd62326a2093..d7a68debdb9ac 100644 --- a/tests/realtikvtest/importintotest4/BUILD.bazel +++ b/tests/realtikvtest/importintotest4/BUILD.bazel @@ -13,7 +13,7 @@ go_test( deps = [ "//br/pkg/lightning/config", "//pkg/config", - "//pkg/disttask/framework/dispatcher", + "//pkg/disttask/framework/scheduler", "//pkg/disttask/framework/storage", "//pkg/disttask/importinto", "//pkg/executor/importer", diff --git a/tests/realtikvtest/importintotest4/global_sort_test.go b/tests/realtikvtest/importintotest4/global_sort_test.go index 88be1e6e59aa7..65ef869d86e3a 100644 --- a/tests/realtikvtest/importintotest4/global_sort_test.go +++ b/tests/realtikvtest/importintotest4/global_sort_test.go @@ -25,7 +25,7 @@ import ( "time" "github.com/fsouza/fake-gcs-server/fakestorage" - "github.com/pingcap/tidb/pkg/disttask/framework/dispatcher" + "github.com/pingcap/tidb/pkg/disttask/framework/scheduler" "github.com/pingcap/tidb/pkg/disttask/framework/storage" "github.com/pingcap/tidb/pkg/disttask/importinto" "github.com/pingcap/tidb/pkg/executor/importer" @@ -61,7 +61,7 @@ func (s *mockGCSSuite) TestGlobalSortBasic() { s.tk.MustExec(`create table t (a bigint primary key, b varchar(100), c varchar(100), d int, key(a), key(c,d), key(d));`) s.enableFailpoint("github.com/pingcap/tidb/pkg/parser/ast/forceRedactURL", "return(true)") - s.enableFailpoint("github.com/pingcap/tidb/pkg/disttask/framework/dispatcher/WaitCleanUpFinished", "return()") + s.enableFailpoint("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished", "return()") sortStorageURI := fmt.Sprintf("gs://sorted/import?endpoint=%s&access-key=aaaaaa&secret-access-key=bbbbbb", gcsEndpoint) importSQL := fmt.Sprintf(`import into t FROM 'gs://gs-basic/t.*.csv?endpoint=%s' @@ -76,7 +76,7 @@ func (s *mockGCSSuite) TestGlobalSortBasic() { )) // check all sorted data cleaned up - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished _, files, err := s.server.ListObjectsWithOptions("sorted", fakestorage.ListOptions{Prefix: "import"}) s.NoError(err) @@ -105,7 +105,7 @@ func (s *mockGCSSuite) TestGlobalSortBasic() { "1 foo1 bar1 123", "2 foo2 bar2 456", "3 foo3 bar3 789", "4 foo4 bar4 123", "5 foo5 bar5 223", "6 foo6 bar6 323", )) - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished // failed task, should clean up all sorted data too. s.enableFailpoint("github.com/pingcap/tidb/pkg/disttask/importinto/failWhenDispatchWriteIngestSubtask", "return(true)") @@ -120,7 +120,7 @@ func (s *mockGCSSuite) TestGlobalSortBasic() { return task.State == "failed" }, 30*time.Second, 300*time.Millisecond) // check all sorted data cleaned up - <-dispatcher.WaitCleanUpFinished + <-scheduler.WaitCleanUpFinished _, files, err = s.server.ListObjectsWithOptions("sorted", fakestorage.ListOptions{Prefix: "import"}) s.NoError(err)