diff --git a/pkg/ddl/backfilling_clean_s3.go b/pkg/ddl/backfilling_clean_s3.go index 5238b14ec38e9..e88a0804a882c 100644 --- a/pkg/ddl/backfilling_clean_s3.go +++ b/pkg/ddl/backfilling_clean_s3.go @@ -40,15 +40,15 @@ func newBackfillCleanUpS3() dispatcher.CleanUpRoutine { // CleanUp implements the CleanUpRoutine.CleanUp interface. func (*BackfillCleanUpS3) CleanUp(ctx context.Context, task *proto.Task) error { - var gTaskMeta BackfillGlobalMeta - if err := json.Unmarshal(task.Meta, &gTaskMeta); err != nil { + var taskMeta BackfillGlobalMeta + if err := json.Unmarshal(task.Meta, &taskMeta); err != nil { return err } // Not use cloud storage, no need to cleanUp. - if len(gTaskMeta.CloudStorageURI) == 0 { + if len(taskMeta.CloudStorageURI) == 0 { return nil } - backend, err := storage.ParseBackend(gTaskMeta.CloudStorageURI, nil) + backend, err := storage.ParseBackend(taskMeta.CloudStorageURI, nil) if err != nil { logutil.Logger(ctx).Warn("failed to parse cloud storage uri", zap.Error(err)) return err @@ -58,19 +58,19 @@ func (*BackfillCleanUpS3) CleanUp(ctx context.Context, task *proto.Task) error { logutil.Logger(ctx).Warn("failed to create cloud storage", zap.Error(err)) return err } - prefix := strconv.Itoa(int(gTaskMeta.Job.ID)) + prefix := strconv.Itoa(int(taskMeta.Job.ID)) err = external.CleanUpFiles(ctx, extStore, prefix) if err != nil { logutil.Logger(ctx).Warn("cannot cleanup cloud storage files", zap.Error(err)) return err } - redactCloudStorageURI(ctx, task, &gTaskMeta) + redactCloudStorageURI(ctx, task, &taskMeta) return nil } func redactCloudStorageURI( ctx context.Context, - gTask *proto.Task, + task *proto.Task, origin *BackfillGlobalMeta, ) { origin.CloudStorageURI = ast.RedactURL(origin.CloudStorageURI) @@ -79,5 +79,5 @@ func redactCloudStorageURI( logutil.Logger(ctx).Warn("failed to marshal task meta", zap.Error(err)) return } - gTask.Meta = metaBytes + task.Meta = metaBytes } diff --git a/pkg/ddl/backfilling_dispatcher.go b/pkg/ddl/backfilling_dispatcher.go index 1e2036a9b4fc2..8caad67d23cdb 100644 --- a/pkg/ddl/backfilling_dispatcher.go +++ b/pkg/ddl/backfilling_dispatcher.go @@ -69,18 +69,18 @@ func (*BackfillingDispatcherExt) OnTick(_ context.Context, _ *proto.Task) { func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( ctx context.Context, taskHandle dispatcher.TaskHandle, - gTask *proto.Task, + task *proto.Task, serverInfo []*infosync.ServerInfo, nextStep proto.Step, ) (taskMeta [][]byte, err error) { logger := logutil.BgLogger().With( - zap.Stringer("type", gTask.Type), - zap.Int64("task-id", gTask.ID), - zap.String("curr-step", StepStr(gTask.Step)), + zap.Stringer("type", task.Type), + zap.Int64("task-id", task.ID), + zap.String("curr-step", StepStr(task.Step)), zap.String("next-step", StepStr(nextStep)), ) var backfillMeta BackfillGlobalMeta - if err := json.Unmarshal(gTask.Meta, &backfillMeta); err != nil { + if err := json.Unmarshal(task.Meta, &backfillMeta); err != nil { return nil, err } job := &backfillMeta.Job @@ -98,13 +98,13 @@ func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( } return generateNonPartitionPlan(dsp.d, tblInfo, job, dsp.GlobalSort, len(serverInfo)) case StepMergeSort: - res, err := generateMergePlan(taskHandle, gTask, logger) + res, err := generateMergePlan(taskHandle, task, logger) if err != nil { return nil, err } if len(res) > 0 { backfillMeta.UseMergeSort = true - if err := updateMeta(gTask, &backfillMeta); err != nil { + if err := updateMeta(task, &backfillMeta); err != nil { return nil, err } } @@ -128,7 +128,7 @@ func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( return generateGlobalSortIngestPlan( ctx, taskHandle, - gTask, + task, job.ID, backfillMeta.CloudStorageURI, prevStep, @@ -140,12 +140,12 @@ func (dsp *BackfillingDispatcherExt) OnNextSubtasksBatch( } } -func updateMeta(gTask *proto.Task, taskMeta *BackfillGlobalMeta) error { +func updateMeta(task *proto.Task, taskMeta *BackfillGlobalMeta) error { bs, err := json.Marshal(taskMeta) if err != nil { return errors.Trace(err) } - gTask.Meta = bs + task.Meta = bs return nil } diff --git a/pkg/ddl/backfilling_dispatcher_test.go b/pkg/ddl/backfilling_dispatcher_test.go index 29a362649a022..7db98ff42bc69 100644 --- a/pkg/ddl/backfilling_dispatcher_test.go +++ b/pkg/ddl/backfilling_dispatcher_test.go @@ -61,17 +61,17 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { "PARTITION p1 VALUES LESS THAN (100),\n" + "PARTITION p2 VALUES LESS THAN (1000),\n" + "PARTITION p3 VALUES LESS THAN MAXVALUE\n);") - gTask := createAddIndexGlobalTask(t, dom, "test", "tp1", proto.Backfill, false) + task := createAddIndexTask(t, dom, "test", "tp1", proto.Backfill, false) tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("tp1")) require.NoError(t, err) tblInfo := tbl.Meta() // 1.1 OnNextSubtasksBatch - gTask.Step = dsp.GetNextStep(gTask) - require.Equal(t, ddl.StepReadIndex, gTask.Step) - serverInfos, _, err := dsp.GetEligibleInstances(context.Background(), gTask) + task.Step = dsp.GetNextStep(task) + require.Equal(t, ddl.StepReadIndex, task.Step) + serverInfos, _, err := dsp.GetEligibleInstances(context.Background(), task) require.NoError(t, err) - metas, err := dsp.OnNextSubtasksBatch(context.Background(), nil, gTask, serverInfos, gTask.Step) + metas, err := dsp.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 { @@ -81,22 +81,22 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { } // 1.2 test partition table OnNextSubtasksBatch after StepReadIndex - gTask.State = proto.TaskStateRunning - gTask.Step = dsp.GetNextStep(gTask) - require.Equal(t, proto.StepDone, gTask.Step) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, gTask, serverInfos, gTask.Step) + task.State = proto.TaskStateRunning + task.Step = dsp.GetNextStep(task) + require.Equal(t, proto.StepDone, task.Step) + metas, err = dsp.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, gTask) + err = dsp.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)") - gTask = createAddIndexGlobalTask(t, dom, "test", "t1", proto.Backfill, false) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, gTask, serverInfos, gTask.Step) + task = createAddIndexTask(t, dom, "test", "t1", proto.Backfill, false) + metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, 0, len(metas)) // 2.2 non empty table. @@ -105,18 +105,18 @@ func TestBackfillingDispatcherLocalMode(t *testing.T) { tk.MustExec("insert into t2 values (), (), (), (), (), ()") tk.MustExec("insert into t2 values (), (), (), (), (), ()") tk.MustExec("insert into t2 values (), (), (), (), (), ()") - gTask = createAddIndexGlobalTask(t, dom, "test", "t2", proto.Backfill, false) + task = createAddIndexTask(t, dom, "test", "t2", proto.Backfill, false) // 2.2.1 stepInit - gTask.Step = dsp.GetNextStep(gTask) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, gTask, serverInfos, gTask.Step) + task.Step = dsp.GetNextStep(task) + metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, 1, len(metas)) - require.Equal(t, ddl.StepReadIndex, gTask.Step) + require.Equal(t, ddl.StepReadIndex, task.Step) // 2.2.2 StepReadIndex - gTask.State = proto.TaskStateRunning - gTask.Step = dsp.GetNextStep(gTask) - require.Equal(t, proto.StepDone, gTask.Step) - metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, gTask, serverInfos, gTask.Step) + task.State = proto.TaskStateRunning + task.Step = dsp.GetNextStep(task) + require.Equal(t, proto.StepDone, task.Step) + metas, err = dsp.OnNextSubtasksBatch(context.Background(), nil, task, serverInfos, task.Step) require.NoError(t, err) require.Equal(t, 0, len(metas)) } @@ -162,7 +162,7 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { tk.MustExec("insert into t1 values (), (), (), (), (), ()") tk.MustExec("insert into t1 values (), (), (), (), (), ()") tk.MustExec("insert into t1 values (), (), (), (), (), ()") - task := createAddIndexGlobalTask(t, dom, "test", "t1", proto.Backfill, true) + task := createAddIndexTask(t, dom, "test", "t1", proto.Backfill, true) dsp := dspManager.MockDispatcher(task) ext, err := ddl.NewBackfillingDispatcherExt(dom.DDL()) @@ -170,7 +170,7 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { ext.(*ddl.BackfillingDispatcherExt).GlobalSort = true dsp.Extension = ext - taskID, err := mgr.AddNewGlobalTask(ctx, task.Key, proto.Backfill, 1, task.Meta) + 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) @@ -185,9 +185,9 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { // update task/subtask, and finish subtask, so we can go to next stage subtasks := make([]*proto.Subtask, 0, len(subtaskMetas)) for _, m := range subtaskMetas { - subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", m)) + subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", 1, m)) } - _, err = mgr.UpdateGlobalTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) + _, err = mgr.UpdateTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) require.NoError(t, err) gotSubtasks, err := mgr.GetSubtasksForImportInto(ctx, taskID, ddl.StepReadIndex) require.NoError(t, err) @@ -210,7 +210,7 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { sortStepMetaBytes, err := json.Marshal(sortStepMeta) require.NoError(t, err) for _, s := range gotSubtasks { - require.NoError(t, mgr.FinishSubtask(ctx, s.SchedulerID, s.ID, sortStepMetaBytes)) + require.NoError(t, mgr.FinishSubtask(ctx, s.ExecID, s.ID, sortStepMetaBytes)) } // 2. to merge-sort stage. require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/forceMergeSort", `return()`)) @@ -226,9 +226,9 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { // update meta, same as import into. subtasks = make([]*proto.Subtask, 0, len(subtaskMetas)) for _, m := range subtaskMetas { - subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", m)) + subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", 1, m)) } - _, err = mgr.UpdateGlobalTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) + _, err = mgr.UpdateTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) require.NoError(t, err) gotSubtasks, err = mgr.GetSubtasksForImportInto(ctx, taskID, task.Step) require.NoError(t, err) @@ -249,7 +249,7 @@ func TestBackfillingDispatcherGlobalSortMode(t *testing.T) { mergeSortStepMetaBytes, err := json.Marshal(mergeSortStepMeta) require.NoError(t, err) for _, s := range gotSubtasks { - require.NoError(t, mgr.FinishSubtask(ctx, s.SchedulerID, s.ID, mergeSortStepMetaBytes)) + require.NoError(t, mgr.FinishSubtask(ctx, s.ExecID, s.ID, mergeSortStepMetaBytes)) } // 3. to write&ingest stage. require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/mockWriteIngest", "return(true)")) @@ -289,7 +289,7 @@ func TestGetNextStep(t *testing.T) { } } -func createAddIndexGlobalTask(t *testing.T, +func createAddIndexTask(t *testing.T, dom *domain.Domain, dbName, tblName string, @@ -322,18 +322,18 @@ func createAddIndexGlobalTask(t *testing.T, taskMeta.CloudStorageURI = "gs://sort-bucket" } - gTaskMetaBytes, err := json.Marshal(taskMeta) + taskMetaBytes, err := json.Marshal(taskMeta) require.NoError(t, err) - gTask := &proto.Task{ + task := &proto.Task{ ID: time.Now().UnixMicro(), Type: taskType, Step: proto.StepInit, State: proto.TaskStatePending, - Meta: gTaskMetaBytes, + Meta: taskMetaBytes, StartTime: time.Now(), StateUpdateTime: time.Now(), } - return gTask + return task } diff --git a/pkg/ddl/constant.go b/pkg/ddl/constant.go index 8b3757bf6f7b9..e09da2c8f6c2b 100644 --- a/pkg/ddl/constant.go +++ b/pkg/ddl/constant.go @@ -57,12 +57,17 @@ const ( exec_expired timestamp, state varchar(64) not null, checkpoint longblob not null, + concurrency int, + create_time timestamp, start_time bigint, state_update_time bigint, + end_time TIMESTAMP, meta longblob, error BLOB, summary json, - key idx_task_key(task_key))` + key idx_task_key(task_key), + key idx_exec_id(exec_id) + )` // BackgroundSubtaskHistoryTableSQL is the CREATE TABLE SQL of `tidb_background_subtask_history`. BackgroundSubtaskHistoryTableSQL = `create table tidb_background_subtask_history ( id bigint not null auto_increment primary key, @@ -75,8 +80,11 @@ const ( exec_expired timestamp, state varchar(64) not null, checkpoint longblob not null, + concurrency int, + create_time timestamp, start_time bigint, state_update_time bigint, + end_time TIMESTAMP, meta longblob, error BLOB, summary json, diff --git a/pkg/ddl/index.go b/pkg/ddl/index.go index 5a1bcc6292a03..077a0e764b212 100644 --- a/pkg/ddl/index.go +++ b/pkg/ddl/index.go @@ -1966,7 +1966,7 @@ func (w *worker) addTableIndex(t table.Table, reorgInfo *reorgInfo) error { // TODO: Support typeAddIndexMergeTmpWorker. if reorgInfo.ReorgMeta.IsDistReorg && !reorgInfo.mergingTmpIdx { if reorgInfo.ReorgMeta.ReorgTp == model.ReorgTypeLitMerge { - err := w.executeDistGlobalTask(reorgInfo) + err := w.executeDistTask(reorgInfo) if err != nil { return err } @@ -2048,7 +2048,7 @@ var MockDMLExecutionOnDDLPaused func() // TestSyncChan is used to sync the test. var TestSyncChan = make(chan struct{}) -func (w *worker) executeDistGlobalTask(reorgInfo *reorgInfo) error { +func (w *worker) executeDistTask(reorgInfo *reorgInfo) error { if reorgInfo.mergingTmpIdx { return errors.New("do not support merge index") } @@ -2074,7 +2074,7 @@ func (w *worker) executeDistGlobalTask(reorgInfo *reorgInfo) error { if err != nil { return err } - task, err := taskManager.GetGlobalTaskByKeyWithHistory(w.ctx, taskKey) + task, err := taskManager.GetTaskByKeyWithHistory(w.ctx, taskKey) if err != nil { return err } @@ -2099,7 +2099,7 @@ func (w *worker) executeDistGlobalTask(reorgInfo *reorgInfo) error { if err != nil { return err } - err = handle.WaitGlobalTask(ctx, task.ID) + err = handle.WaitTask(ctx, task.ID) if err := w.isReorgRunnable(reorgInfo.Job.ID, true); err != nil { if dbterror.ErrPausedDDLJob.Equal(err) { logutil.BgLogger().Warn("job paused by user", zap.String("category", "ddl"), zap.Error(err)) @@ -2129,7 +2129,7 @@ func (w *worker) executeDistGlobalTask(reorgInfo *reorgInfo) error { g.Go(func() error { defer close(done) - err := handle.SubmitAndRunGlobalTask(ctx, taskKey, taskType, distPhysicalTableConcurrency, metaData) + err := handle.SubmitAndWaitTask(ctx, taskKey, taskType, distPhysicalTableConcurrency, metaData) failpoint.Inject("pauseAfterDistTaskFinished", func() { MockDMLExecutionOnTaskFinished() }) @@ -2168,7 +2168,7 @@ func (w *worker) executeDistGlobalTask(reorgInfo *reorgInfo) error { if !dbterror.ErrCancelledDDLJob.Equal(err) { return errors.Trace(err) } - if err = handle.CancelGlobalTask(w.ctx, taskKey); err != nil { + if err = handle.CancelTask(w.ctx, taskKey); err != nil { logutil.BgLogger().Error("cancel global task error", zap.String("category", "ddl"), zap.String("task_key", taskKey), zap.Error(err)) // continue to cancel global task. continue @@ -2189,12 +2189,12 @@ func (w *worker) updateJobRowCount(taskKey string, jobID int64) { logutil.BgLogger().Warn("cannot get task manager", zap.String("category", "ddl"), zap.String("task_key", taskKey), zap.Error(err)) return } - gTask, err := taskMgr.GetGlobalTaskByKey(w.ctx, taskKey) - if err != nil || gTask == nil { + task, err := taskMgr.GetTaskByKey(w.ctx, taskKey) + if err != nil || task == nil { logutil.BgLogger().Warn("cannot get global task", zap.String("category", "ddl"), zap.String("task_key", taskKey), zap.Error(err)) return } - rowCount, err := taskMgr.GetSubtaskRowCount(w.ctx, gTask.ID, proto.StepOne) + rowCount, err := taskMgr.GetSubtaskRowCount(w.ctx, task.ID, proto.StepOne) if err != nil { logutil.BgLogger().Warn("cannot get subtask row count", zap.String("category", "ddl"), zap.String("task_key", taskKey), zap.Error(err)) return diff --git a/pkg/disttask/framework/dispatcher/dispatcher.go b/pkg/disttask/framework/dispatcher/dispatcher.go index 7a9b90606e645..b6667ed054c25 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher.go +++ b/pkg/disttask/framework/dispatcher/dispatcher.go @@ -144,7 +144,7 @@ func (*BaseDispatcher) Close() { // refreshTask fetch task state from tidb_global_task table. func (d *BaseDispatcher) refreshTask() error { - newTask, err := d.taskMgr.GetGlobalTaskByID(d.ctx, d.Task.ID) + newTask, err := d.taskMgr.GetTaskByID(d.ctx, d.Task.ID) if err != nil { logutil.Logger(d.logCtx).Error("refresh task failed", zap.Error(err)) return err @@ -172,7 +172,7 @@ func (d *BaseDispatcher) scheduleTask() { } failpoint.Inject("cancelTaskAfterRefreshTask", func(val failpoint.Value) { if val.(bool) && d.Task.State == proto.TaskStateRunning { - err := d.taskMgr.CancelGlobalTask(d.ctx, d.Task.ID) + err := d.taskMgr.CancelTask(d.ctx, d.Task.ID) if err != nil { logutil.Logger(d.logCtx).Error("cancel task failed", zap.Error(err)) } @@ -457,8 +457,8 @@ func (d *BaseDispatcher) ReDispatchSubtasks() error { subtasksOnScheduler[execID] = make([]*proto.Subtask, 0) } for _, subtask := range subtasks { - subtasksOnScheduler[subtask.SchedulerID] = append( - subtasksOnScheduler[subtask.SchedulerID], + subtasksOnScheduler[subtask.ExecID] = append( + subtasksOnScheduler[subtask.ExecID], subtask) } // 4. prepare subtasks that need to rebalance to other nodes. @@ -492,7 +492,7 @@ func (d *BaseDispatcher) ReDispatchSubtasks() error { if ok := deadNodesMap[k]; !ok { if len(v) < averageSubtaskCnt { for i := 0; i < averageSubtaskCnt-len(v) && rebalanceIdx < len(rebalanceSubtasks); i++ { - rebalanceSubtasks[rebalanceIdx].SchedulerID = k + rebalanceSubtasks[rebalanceIdx].ExecID = k rebalanceIdx++ } } @@ -502,7 +502,7 @@ func (d *BaseDispatcher) ReDispatchSubtasks() error { liveNodeIdx := 0 for rebalanceIdx < len(rebalanceSubtasks) { node := d.LiveNodes[liveNodeIdx] - rebalanceSubtasks[rebalanceIdx].SchedulerID = disttaskutil.GenerateExecID(node.IP, node.Port) + rebalanceSubtasks[rebalanceIdx].ExecID = disttaskutil.GenerateExecID(node.IP, node.Port) rebalanceIdx++ liveNodeIdx++ } @@ -530,7 +530,7 @@ func (d *BaseDispatcher) updateTask(taskState proto.TaskState, newSubTasks []*pr } failpoint.Inject("cancelBeforeUpdate", func() { - err := d.taskMgr.CancelGlobalTask(d.ctx, d.Task.ID) + err := d.taskMgr.CancelTask(d.ctx, d.Task.ID) if err != nil { logutil.Logger(d.logCtx).Error("cancel task failed", zap.Error(err)) } @@ -538,7 +538,7 @@ func (d *BaseDispatcher) updateTask(taskState proto.TaskState, newSubTasks []*pr var retryable bool for i := 0; i < retryTimes; i++ { - retryable, err = d.taskMgr.UpdateGlobalTaskAndAddSubTasks(d.ctx, d.Task, newSubTasks, prevState) + retryable, err = d.taskMgr.UpdateTaskAndAddSubTasks(d.ctx, d.Task, newSubTasks, prevState) if err == nil || !retryable { break } @@ -571,7 +571,7 @@ func (d *BaseDispatcher) onErrHandlingStage(receiveErrs []error) error { 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, []byte("{}"))) + subTasks = append(subTasks, proto.NewSubtask(d.Task.Step, d.Task.ID, d.Task.Type, id, int(d.Task.Concurrency), []byte("{}"))) } } return d.updateTask(proto.TaskStateReverting, subTasks, RetrySQLTimes) @@ -594,7 +594,7 @@ func (d *BaseDispatcher) onNextStage() (err error) { zap.Int64("current-step", int64(d.Task.Step)), zap.Int64("next-step", int64(nextStep))) - // 1. Adjust the global task's concurrency. + // 1. Adjust the task's concurrency. if d.Task.State == proto.TaskStatePending { if d.Task.Concurrency == 0 { d.Task.Concurrency = DefaultSubtaskConcurrency @@ -699,7 +699,7 @@ func (d *BaseDispatcher) dispatchSubTask( 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, meta)) + subTasks = append(subTasks, proto.NewSubtask(subtaskStep, d.Task.ID, d.Task.Type, instanceID, int(d.Task.Concurrency), meta)) } failpoint.Inject("cancelBeforeUpdateTask", func() { _ = d.updateTask(proto.TaskStateCancelling, subTasks, RetrySQLTimes) diff --git a/pkg/disttask/framework/dispatcher/dispatcher_manager.go b/pkg/disttask/framework/dispatcher/dispatcher_manager.go index 0f147782153f6..8186a38d031e1 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher_manager.go +++ b/pkg/disttask/framework/dispatcher/dispatcher_manager.go @@ -151,7 +151,7 @@ func (dm *Manager) Inited() bool { return dm.inited } -// dispatchTaskLoop dispatches the global tasks. +// dispatchTaskLoop dispatches the tasks. func (dm *Manager) dispatchTaskLoop() { logutil.BgLogger().Info("dispatch task loop start") ticker := time.NewTicker(checkTaskRunningInterval) @@ -168,7 +168,7 @@ func (dm *Manager) dispatchTaskLoop() { } // TODO: Consider getting these tasks, in addition to the task being worked on.. - tasks, err := dm.taskMgr.GetGlobalTasksInStates( + tasks, err := dm.taskMgr.GetTasksInStates( dm.ctx, proto.TaskStatePending, proto.TaskStateRunning, @@ -181,12 +181,12 @@ func (dm *Manager) dispatchTaskLoop() { break } - // There are currently no global tasks to work on. + // There are currently no tasks to work on. if len(tasks) == 0 { break } for _, task := range tasks { - // This global task is running, so no need to reprocess it. + // This task is running, so no need to reprocess it. if dm.isRunningTask(task.ID) { continue } @@ -224,7 +224,7 @@ func (dm *Manager) failTask(task *proto.Task, err error) { prevState := task.State task.State = proto.TaskStateFailed task.Error = err - if _, err2 := dm.taskMgr.UpdateGlobalTaskAndAddSubTasks(dm.ctx, task, nil, prevState); err2 != nil { + if _, err2 := dm.taskMgr.UpdateTaskAndAddSubTasks(dm.ctx, task, nil, prevState); err2 != nil { logutil.BgLogger().Warn("failed to update task state to failed", zap.Int64("task-id", task.ID), zap.Error(err2)) } @@ -318,7 +318,7 @@ func (dm *Manager) doCleanUpRoutine() { if cnt != 0 { logutil.BgLogger().Info("clean up nodes in framework meta since nodes shutdown", zap.Int("cnt", cnt)) } - tasks, err := dm.taskMgr.GetGlobalTasksInStates( + tasks, err := dm.taskMgr.GetTasksInStates( dm.ctx, proto.TaskStateFailed, proto.TaskStateReverted, diff --git a/pkg/disttask/framework/dispatcher/dispatcher_manager_test.go b/pkg/disttask/framework/dispatcher/dispatcher_manager_test.go index 6b0b548f3e071..dfec0c8127368 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher_manager_test.go +++ b/pkg/disttask/framework/dispatcher/dispatcher_manager_test.go @@ -46,14 +46,14 @@ func TestCleanUpRoutine(t *testing.T) { dsp.Start() defer dsp.Stop() require.NoError(t, mgr.StartManager(ctx, ":4000", "background")) - taskID, err := mgr.AddNewGlobalTask(ctx, "test", proto.TaskTypeExample, 1, nil) + taskID, err := mgr.CreateTask(ctx, "test", proto.TaskTypeExample, 1, nil) require.NoError(t, err) checkTaskRunningCnt := func() []*proto.Task { var tasks []*proto.Task require.Eventually(t, func() bool { var err error - tasks, err = mgr.GetGlobalTasksInStates(ctx, proto.TaskStateRunning) + tasks, err = mgr.GetTasksInStates(ctx, proto.TaskStateRunning) require.NoError(t, err) return len(tasks) == 1 }, time.Second, 50*time.Millisecond) @@ -76,7 +76,7 @@ func TestCleanUpRoutine(t *testing.T) { } dsp.DoCleanUpRoutine() require.Eventually(t, func() bool { - tasks, err := mgr.GetGlobalTasksFromHistoryInStates(ctx, proto.TaskStateSucceed) + tasks, err := mgr.GetTasksFromHistoryInStates(ctx, proto.TaskStateSucceed) require.NoError(t, err) return len(tasks) != 0 }, time.Second*10, time.Millisecond*300) diff --git a/pkg/disttask/framework/dispatcher/dispatcher_test.go b/pkg/disttask/framework/dispatcher/dispatcher_test.go index 0e7927dfe6d37..5a252f598960b 100644 --- a/pkg/disttask/framework/dispatcher/dispatcher_test.go +++ b/pkg/disttask/framework/dispatcher/dispatcher_test.go @@ -59,7 +59,7 @@ func getTestDispatcherExt(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, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + func(_ context.Context, _ dispatcher.TaskHandle, _ *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { return nil, nil }, ).AnyTimes() @@ -194,11 +194,11 @@ func TestGetInstance(t *testing.T) { // server ids: uuid0, uuid1 // subtask instance ids: uuid1 subtask := &proto.Subtask{ - Type: proto.TaskTypeExample, - TaskID: task.ID, - SchedulerID: serverIDs[1], + Type: proto.TaskTypeExample, + TaskID: task.ID, + ExecID: serverIDs[1], } - err = mgr.AddNewSubTask(ctx, task.ID, proto.StepInit, subtask.SchedulerID, nil, subtask.Type, true) + err = mgr.CreateSubTask(ctx, task.ID, proto.StepInit, subtask.ExecID, nil, subtask.Type, true) require.NoError(t, err) instanceIDs, err = dsp.GetAllSchedulerIDs(ctx, task) require.NoError(t, err) @@ -206,11 +206,11 @@ func TestGetInstance(t *testing.T) { // server ids: uuid0, uuid1 // subtask instance ids: uuid0, uuid1 subtask = &proto.Subtask{ - Type: proto.TaskTypeExample, - TaskID: task.ID, - SchedulerID: serverIDs[0], + Type: proto.TaskTypeExample, + TaskID: task.ID, + ExecID: serverIDs[0], } - err = mgr.AddNewSubTask(ctx, task.ID, proto.StepInit, subtask.SchedulerID, nil, subtask.Type, true) + err = mgr.CreateSubTask(ctx, task.ID, proto.StepInit, subtask.ExecID, nil, subtask.Type, true) require.NoError(t, err) instanceIDs, err = dsp.GetAllSchedulerIDs(ctx, task) require.NoError(t, err) @@ -242,20 +242,20 @@ func TestTaskFailInManager(t *testing.T) { defer dspManager.Stop() // unknown task type - taskID, err := mgr.AddNewGlobalTask(ctx, "test", "test-type", 1, nil) + taskID, err := mgr.CreateTask(ctx, "test", "test-type", 1, nil) require.NoError(t, err) require.Eventually(t, func() bool { - task, err := mgr.GetGlobalTaskByID(ctx, taskID) + task, err := mgr.GetTaskByID(ctx, taskID) require.NoError(t, err) return task.State == proto.TaskStateFailed && strings.Contains(task.Error.Error(), "unknown task type") }, time.Second*10, time.Millisecond*300) // dispatcher init error - taskID, err = mgr.AddNewGlobalTask(ctx, "test2", proto.TaskTypeExample, 1, nil) + taskID, err = mgr.CreateTask(ctx, "test2", proto.TaskTypeExample, 1, nil) require.NoError(t, err) require.Eventually(t, func() bool { - task, err := mgr.GetGlobalTaskByID(ctx, taskID) + task, err := mgr.GetTaskByID(ctx, taskID) require.NoError(t, err) return task.State == proto.TaskStateFailed && strings.Contains(task.Error.Error(), "mock dispatcher init error") @@ -311,7 +311,7 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, var tasks []*proto.Task require.Eventually(t, func() bool { var err error - tasks, err = mgr.GetGlobalTasksInStates(ctx, proto.TaskStateRunning) + tasks, err = mgr.GetTasksInStates(ctx, proto.TaskStateRunning) require.NoError(t, err) return len(tasks) == taskCnt }, time.Second, 50*time.Millisecond) @@ -332,7 +332,7 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, // Mock add tasks. taskIDs := make([]int64, 0, taskCnt) for i := 0; i < taskCnt; i++ { - taskID, err := mgr.AddNewGlobalTask(ctx, fmt.Sprintf("%d", i), proto.TaskTypeExample, 0, nil) + taskID, err := mgr.CreateTask(ctx, fmt.Sprintf("%d", i), proto.TaskTypeExample, 0, nil) require.NoError(t, err) taskIDs = append(taskIDs, taskID) } @@ -342,7 +342,7 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, checkSubtaskCnt(tasks, taskIDs) // test parallelism control if taskCnt == 1 { - taskID, err := mgr.AddNewGlobalTask(ctx, fmt.Sprintf("%d", taskCnt), proto.TaskTypeExample, 0, nil) + taskID, err := mgr.CreateTask(ctx, fmt.Sprintf("%d", taskCnt), proto.TaskTypeExample, 0, nil) require.NoError(t, err) checkGetRunningTaskCnt(taskCnt) // Clean the task. @@ -354,12 +354,12 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, checkGetTaskState := func(expectedState proto.TaskState) { i := 0 for ; i < cnt; i++ { - tasks, err := mgr.GetGlobalTasksInStates(ctx, expectedState) + tasks, err := mgr.GetTasksInStates(ctx, expectedState) require.NoError(t, err) if len(tasks) == taskCnt { break } - historyTasks, err := mgr.GetGlobalTasksFromHistoryInStates(ctx, expectedState) + historyTasks, err := mgr.GetTasksFromHistoryInStates(ctx, expectedState) require.NoError(t, err) if len(tasks)+len(historyTasks) == taskCnt { break @@ -385,7 +385,7 @@ func checkDispatch(t *testing.T, taskCnt int, isSucc, isCancel, isSubtaskCancel, if isCancel { for i := 1; i <= taskCnt; i++ { - err = mgr.CancelGlobalTask(ctx, int64(i)) + err = mgr.CancelTask(ctx, int64(i)) require.NoError(t, err) } } else if isPauseAndResume { diff --git a/pkg/disttask/framework/dispatcher/interface.go b/pkg/disttask/framework/dispatcher/interface.go index 75d062c228875..bed5dfe594cec 100644 --- a/pkg/disttask/framework/dispatcher/interface.go +++ b/pkg/disttask/framework/dispatcher/interface.go @@ -25,14 +25,14 @@ import ( // TaskManager defines the interface to access task table. type TaskManager interface { - GetGlobalTasksInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) - GetGlobalTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error) - UpdateGlobalTaskAndAddSubTasks(ctx context.Context, gTask *proto.Task, subtasks []*proto.Subtask, prevState proto.TaskState) (bool, error) + GetTasksInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) + GetTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error) + UpdateTaskAndAddSubTasks(ctx context.Context, task *proto.Task, subtasks []*proto.Subtask, prevState proto.TaskState) (bool, error) GCSubtasks(ctx context.Context) error GetAllNodes(ctx context.Context) ([]string, error) CleanUpMeta(ctx context.Context, nodes []string) error TransferTasks2History(ctx context.Context, tasks []*proto.Task) error - CancelGlobalTask(ctx context.Context, taskID int64) error + CancelTask(ctx context.Context, taskID int64) error PauseTask(ctx context.Context, taskKey string) (bool, error) GetSubtaskInStatesCnt(ctx context.Context, taskID int64, states ...interface{}) (int64, error) ResumeSubtasks(ctx context.Context, taskID int64) error @@ -60,7 +60,7 @@ type Extension interface { OnTick(ctx context.Context, task *proto.Task) // OnNextSubtasksBatch is used to generate batch of subtasks for next stage - // NOTE: don't change gTask.State inside, framework will manage it. + // 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. @@ -125,7 +125,8 @@ func ClearDispatcherFactory() { // CleanUpRoutine is used for the framework to do some clean up work if the task is finished. type CleanUpRoutine interface { - // CleanUp do the clean up work. + // CleanUp do the cleanup work. + // task.Meta can be updated here, such as redacting some sensitive info. CleanUp(ctx context.Context, task *proto.Task) error } type cleanUpFactoryFn func() CleanUpRoutine diff --git a/pkg/disttask/framework/dispatcher/main_test.go b/pkg/disttask/framework/dispatcher/main_test.go index 160acfeca215d..ca8b90c315026 100644 --- a/pkg/disttask/framework/dispatcher/main_test.go +++ b/pkg/disttask/framework/dispatcher/main_test.go @@ -24,7 +24,7 @@ import ( // DispatcherForTest exports for testing. type DispatcherManagerForTest interface { GetRunningTaskCnt() int - DelRunningTask(globalTaskID int64) + DelRunningTask(id int64) DoCleanUpRoutine() } @@ -34,8 +34,8 @@ func (dm *Manager) GetRunningTaskCnt() int { } // DelRunningGTask implements Dispatcher.DelRunningGTask interface. -func (dm *Manager) DelRunningTask(globalTaskID int64) { - dm.delRunningTask(globalTaskID) +func (dm *Manager) DelRunningTask(id int64) { + dm.delRunningTask(id) } // DoCleanUpRoutine implements Dispatcher.DoCleanUpRoutine interface. diff --git a/pkg/disttask/framework/dispatcher/rebalance_test.go b/pkg/disttask/framework/dispatcher/rebalance_test.go index 3794c81ffb7bc..b9a41c6c2dd60 100644 --- a/pkg/disttask/framework/dispatcher/rebalance_test.go +++ b/pkg/disttask/framework/dispatcher/rebalance_test.go @@ -68,7 +68,7 @@ func scaleTest(t *testing.T, return strings.Compare(i, j) }) slices.SortFunc(testCase.subtasks, func(i, j *proto.Subtask) int { - return strings.Compare(i.SchedulerID, j.SchedulerID) + return strings.Compare(i.ExecID, j.ExecID) }) require.Equal(t, testCase.expectedTaskNodes, dsp.TaskNodes) require.Equal(t, testCase.expectedSubtasks, testCase.subtasks) @@ -93,7 +93,7 @@ func balanceTest(t *testing.T, return strings.Compare(i, j) }) slices.SortFunc(testCase.subtasks, func(i, j *proto.Subtask) int { - return strings.Compare(i.SchedulerID, j.SchedulerID) + return strings.Compare(i.ExecID, j.ExecID) }) require.Equal(t, testCase.expectedSubtasks, testCase.subtasks) } @@ -112,202 +112,202 @@ func TestScaleOutNodes(t *testing.T) { // 1. scale out from 1 node to 2 nodes. 4 subtasks. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}}, []string{"1.1.1.1:4000"}, []string{}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, }, // 2. scale out from 1 node to 2 nodes. 3 subtasks. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}}, []string{"1.1.1.1:4000"}, []string{}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}}, }, // 3. scale out from 2 nodes to 4 nodes. 4 subtasks. { []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{}, []string{"1.1.1.1:4000", "1.1.1.2:4000", "1.1.1.3:4000", "1.1.1.4:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}}, }, // 4. scale out from 2 nodes to 4 nodes. 9 subtasks. { []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{}, []string{"1.1.1.1:4000", "1.1.1.2:4000", "1.1.1.3:4000", "1.1.1.4:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}, - {SchedulerID: "1.1.1.4:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}, + {ExecID: "1.1.1.4:4000"}}, }, // 5. scale out from 2 nodes to 3 nodes. { []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{}, []string{"1.1.1.1:4000", "1.1.1.2:4000", "1.1.1.3:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}}, }, // 6. scale out from 1 node to another 2 node. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}}, []string{"1.1.1.1:4000"}, []string{"1.1.1.1:4000"}, []string{"1.1.1.2:4000", "1.1.1.3:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}}, }, // 7. scale out from tidb1, tidb2 to tidb2, tidb3. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.1:4000"}, []string{"1.1.1.2:4000", "1.1.1.3:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}}, }, // 8. scale from tidb1, tidb2 to tidb3, tidb4. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.3:4000", "1.1.1.4:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}, - {SchedulerID: "1.1.1.4:4000"}}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}, + {ExecID: "1.1.1.4:4000"}}, }, // 9. scale from tidb1, tidb2 to tidb2, tidb3, tidb4. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.1:4000"}, []string{"1.1.1.2:4000", "1.1.1.3:4000", "1.1.1.4:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}}, }, // 10. scale form tidb1, tidb2 to tidb2, tidb3, tidb4. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.1:4000"}, []string{"1.1.1.2:4000", "1.1.1.3:4000", "1.1.1.4:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}}, }, // 11. scale from tidb1(2 subtasks), tidb2(3 subtasks), tidb3(0 subtasks) to tidb1, tidb3, tidb4, tidb5, tidb6. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}, {IP: "1.1.1.5", Port: 4000}, {IP: "1.1.1.6", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.2:4000"}, []string{"1.1.1.1:4000", "1.1.1.3:4000", "1.1.1.4:4000", "1.1.1.5:4000", "1.1.1.6:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}, - {SchedulerID: "1.1.1.5:4000"}, - {SchedulerID: "1.1.1.6:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}, + {ExecID: "1.1.1.5:4000"}, + {ExecID: "1.1.1.6:4000"}}, }, } for i, testCase := range testCases { @@ -329,50 +329,50 @@ func TestScaleInNodes(t *testing.T) { // 1. scale in from tidb1, tidb2 to tidb1. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.2:4000"}, []string{"1.1.1.1:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, }, // 2. scale in from tidb1, tidb2 to tidb3. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.3", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.3:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}}, }, // 5. scale in from 10 nodes to 2 nodes. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}, - {SchedulerID: "1.1.1.5:4000"}, - {SchedulerID: "1.1.1.6:4000"}, - {SchedulerID: "1.1.1.7:4000"}, - {SchedulerID: "1.1.1.8:4000"}, - {SchedulerID: "1.1.1.9:4000"}, - {SchedulerID: "1.1.1.10:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}, + {ExecID: "1.1.1.5:4000"}, + {ExecID: "1.1.1.6:4000"}, + {ExecID: "1.1.1.7:4000"}, + {ExecID: "1.1.1.8:4000"}, + {ExecID: "1.1.1.9:4000"}, + {ExecID: "1.1.1.10:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}}, []string{ "1.1.1.1:4000", @@ -396,46 +396,46 @@ func TestScaleInNodes(t *testing.T) { "1.1.1.10:4000"}, []string{"1.1.1.2:4000", "1.1.1.3:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}, }, }, // 6. scale in from 1 node with 10 subtasks, 1 node with 1 subtasks to 1 node. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []string{"1.1.1.2:4000"}, []string{"1.1.1.1:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, }, } for i, testCase := range testCases { @@ -457,104 +457,104 @@ func TestRebalanceWithoutScale(t *testing.T) { // 1. from tidb1:1, tidb2:3 to tidb1:2, tidb2:2 { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, }, // 2. from tidb1:3, tidb2:2, tidb3:1 to tidb1:2, tidb2:2, tidb3:2 { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000", "1.1.1.3:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.3:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.3:4000"}}, }, // 3. from tidb1: 0, tidb2: 5 to tidb1: 3, tidb2: 2 { []*proto.Subtask{ - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, }, // 4. from tidb1:5, tidb2:0, tidb3:0, tidb4:0, tidb5:0, tidb6:0 to 1,1,1,1,1,0. { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}, {IP: "1.1.1.3", Port: 4000}, {IP: "1.1.1.4", Port: 4000}, {IP: "1.1.1.5", Port: 4000}, {IP: "1.1.1.6", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000", "1.1.1.3:4000", "1.1.1.4:4000", "1.1.1.5:4000", "1.1.1.6:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.3:4000"}, - {SchedulerID: "1.1.1.4:4000"}, - {SchedulerID: "1.1.1.5:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.3:4000"}, + {ExecID: "1.1.1.4:4000"}, + {ExecID: "1.1.1.5:4000"}}, }, // 5. no balance needed. tidb1:2, tidb2:3 { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, }, // 6. no balance needed. tidb1:2, tidb2:2 { []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, []*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}, {IP: "1.1.1.2", Port: 4000}}, []string{"1.1.1.1:4000", "1.1.1.2:4000"}, []*proto.Subtask{ - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.1:4000"}, - {SchedulerID: "1.1.1.2:4000"}, - {SchedulerID: "1.1.1.2:4000"}}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.1:4000"}, + {ExecID: "1.1.1.2:4000"}, + {ExecID: "1.1.1.2:4000"}}, }, } for i, testCase := range testCases { diff --git a/pkg/disttask/framework/framework_test.go b/pkg/disttask/framework/framework_test.go index 78839f511228c..1a1318d39bc53 100644 --- a/pkg/disttask/framework/framework_test.go +++ b/pkg/disttask/framework/framework_test.go @@ -313,7 +313,7 @@ func TestFrameworkCleanUpRoutine(t *testing.T) { <-dispatcher.WaitCleanUpFinished mgr, err := storage.GetTaskManager() require.NoError(t, err) - tasks, err := mgr.GetGlobalTaskByKeyWithHistory(ctx, "key1") + tasks, err := mgr.GetTaskByKeyWithHistory(ctx, "key1") require.NoError(t, err) require.NotEmpty(t, tasks) distContext.Close() diff --git a/pkg/disttask/framework/handle/handle.go b/pkg/disttask/framework/handle/handle.go index 018c707f86807..502d8ef6da9a7 100644 --- a/pkg/disttask/framework/handle/handle.go +++ b/pkg/disttask/framework/handle/handle.go @@ -31,39 +31,39 @@ var ( checkTaskFinishInterval = 300 * time.Millisecond ) -// SubmitGlobalTask submits a global task. -func SubmitGlobalTask(ctx context.Context, taskKey string, taskType proto.TaskType, concurrency int, taskMeta []byte) (*proto.Task, error) { - globalTaskManager, err := storage.GetTaskManager() +// SubmitTask submits a task. +func SubmitTask(ctx context.Context, taskKey string, taskType proto.TaskType, concurrency int, taskMeta []byte) (*proto.Task, error) { + taskManager, err := storage.GetTaskManager() if err != nil { return nil, err } - globalTask, err := globalTaskManager.GetGlobalTaskByKey(ctx, taskKey) + task, err := taskManager.GetTaskByKey(ctx, taskKey) if err != nil { return nil, err } - if globalTask == nil { - taskID, err := globalTaskManager.AddNewGlobalTask(ctx, taskKey, taskType, concurrency, taskMeta) + if task == nil { + taskID, err := taskManager.CreateTask(ctx, taskKey, taskType, concurrency, taskMeta) if err != nil { return nil, err } - globalTask, err = globalTaskManager.GetGlobalTaskByID(ctx, taskID) + task, err = taskManager.GetTaskByID(ctx, taskID) if err != nil { return nil, err } - if globalTask == nil { - return nil, errors.Errorf("cannot find global task with ID %d", taskID) + if task == nil { + return nil, errors.Errorf("cannot find task with ID %d", taskID) } - metrics.UpdateMetricsForAddTask(globalTask) + metrics.UpdateMetricsForAddTask(task) } - return globalTask, nil + return task, nil } -// WaitGlobalTask waits for a global task done or paused. +// WaitTask waits for a task done or paused. // this API returns error if task failed or cancelled. -func WaitGlobalTask(ctx context.Context, id int64) error { +func WaitTask(ctx context.Context, id int64) error { logger := logutil.Logger(ctx).With(zap.Int64("task-id", id)) found, err := waitTask(ctx, id, func(t *proto.Task) bool { return t.IsDone() || t.State == proto.TaskStatePaused @@ -76,10 +76,10 @@ func WaitGlobalTask(ctx context.Context, id int64) error { case proto.TaskStateSucceed: return nil case proto.TaskStateReverted: - logger.Error("global task reverted", zap.Error(found.Error)) + logger.Error("task reverted", zap.Error(found.Error)) return found.Error case proto.TaskStatePaused: - logger.Error("global task paused") + logger.Error("task paused") return nil case proto.TaskStateFailed: return errors.Errorf("task stopped with state %s, err %v", found.State, found.Error) @@ -87,18 +87,18 @@ func WaitGlobalTask(ctx context.Context, id int64) error { return nil } -// WaitTaskDoneByKey waits for a global task done by task key. +// WaitTaskDoneByKey waits for a task done by task key. func WaitTaskDoneByKey(ctx context.Context, taskKey string) error { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() if err != nil { return err } - task, err := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) if err != nil { return err } if task == nil { - return errors.Errorf("cannot find global task with key %s", taskKey) + return errors.Errorf("cannot find task with key %s", taskKey) } _, err = waitTask(ctx, task.ID, func(t *proto.Task) bool { return t.IsDone() @@ -107,7 +107,7 @@ func WaitTaskDoneByKey(ctx context.Context, taskKey string) error { } func waitTask(ctx context.Context, id int64, matchFn func(*proto.Task) bool) (*proto.Task, error) { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() if err != nil { return nil, err } @@ -120,13 +120,13 @@ func waitTask(ctx context.Context, id int64, matchFn func(*proto.Task) bool) (*p case <-ctx.Done(): return nil, ctx.Err() case <-ticker.C: - found, err := globalTaskManager.GetTaskByIDWithHistory(ctx, id) + found, err := taskManager.GetTaskByIDWithHistory(ctx, id) if err != nil { - logger.Error("cannot get global task during waiting", zap.Error(err)) + logger.Error("cannot get task during waiting", zap.Error(err)) continue } if found == nil { - return nil, errors.Errorf("cannot find global task with ID %d", id) + return nil, errors.Errorf("cannot find task with ID %d", id) } if matchFn(found) { @@ -136,22 +136,22 @@ func waitTask(ctx context.Context, id int64, matchFn func(*proto.Task) bool) (*p } } -// SubmitAndRunGlobalTask submits a global task and wait for it to finish. -func SubmitAndRunGlobalTask(ctx context.Context, taskKey string, taskType proto.TaskType, concurrency int, taskMeta []byte) error { - globalTask, err := SubmitGlobalTask(ctx, taskKey, taskType, concurrency, taskMeta) +// SubmitAndWaitTask submits a task and wait for it to finish. +func SubmitAndWaitTask(ctx context.Context, taskKey string, taskType proto.TaskType, concurrency int, taskMeta []byte) error { + task, err := SubmitTask(ctx, taskKey, taskType, concurrency, taskMeta) if err != nil { return err } - return WaitGlobalTask(ctx, globalTask.ID) + return WaitTask(ctx, task.ID) } -// CancelGlobalTask cancels a global task. -func CancelGlobalTask(ctx context.Context, taskKey string) error { +// CancelTask cancels a task. +func CancelTask(ctx context.Context, taskKey string) error { taskManager, err := storage.GetTaskManager() if err != nil { return err } - task, err := taskManager.GetGlobalTaskByKey(ctx, taskKey) + task, err := taskManager.GetTaskByKey(ctx, taskKey) if err != nil { return err } @@ -160,7 +160,7 @@ func CancelGlobalTask(ctx context.Context, taskKey string) error { return nil } - return taskManager.CancelGlobalTask(ctx, task.ID) + return taskManager.CancelTask(ctx, task.ID) } // PauseTask pauses a task. diff --git a/pkg/disttask/framework/handle/handle_test.go b/pkg/disttask/framework/handle/handle_test.go index 65e2eaa52c42b..32d299569c27e 100644 --- a/pkg/disttask/framework/handle/handle_test.go +++ b/pkg/disttask/framework/handle/handle_test.go @@ -47,10 +47,10 @@ func TestHandle(t *testing.T) { storage.SetTaskManager(mgr) // no dispatcher registered - err := handle.SubmitAndRunGlobalTask(ctx, "1", proto.TaskTypeExample, 2, []byte("byte")) + err := handle.SubmitAndWaitTask(ctx, "1", proto.TaskTypeExample, 2, []byte("byte")) require.Error(t, err) - task, err := mgr.GetGlobalTaskByID(ctx, 1) + task, err := mgr.GetTaskByID(ctx, 1) require.NoError(t, err) require.Equal(t, int64(1), task.ID) require.Equal(t, "1", task.Key) @@ -61,9 +61,9 @@ func TestHandle(t *testing.T) { require.Equal(t, uint64(2), task.Concurrency) require.Equal(t, []byte("byte"), task.Meta) - require.NoError(t, handle.CancelGlobalTask(ctx, "1")) + require.NoError(t, handle.CancelTask(ctx, "1")) - task, err = handle.SubmitGlobalTask(ctx, "2", proto.TaskTypeExample, 2, []byte("byte")) + task, err = handle.SubmitTask(ctx, "2", proto.TaskTypeExample, 2, []byte("byte")) require.NoError(t, err) require.NoError(t, handle.PauseTask(ctx, "2")) require.NoError(t, handle.ResumeTask(ctx, "2")) diff --git a/pkg/disttask/framework/mock/dispatcher_mock.go b/pkg/disttask/framework/mock/dispatcher_mock.go index 2ebb913abc6af..7c1d03cf94ba5 100644 --- a/pkg/disttask/framework/mock/dispatcher_mock.go +++ b/pkg/disttask/framework/mock/dispatcher_mock.go @@ -138,18 +138,18 @@ func (m *MockTaskManager) EXPECT() *MockTaskManagerMockRecorder { return m.recorder } -// CancelGlobalTask mocks base method. -func (m *MockTaskManager) CancelGlobalTask(arg0 context.Context, arg1 int64) error { +// CancelTask mocks base method. +func (m *MockTaskManager) CancelTask(arg0 context.Context, arg1 int64) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "CancelGlobalTask", arg0, arg1) + ret := m.ctrl.Call(m, "CancelTask", arg0, arg1) ret0, _ := ret[0].(error) return ret0 } -// CancelGlobalTask indicates an expected call of CancelGlobalTask. -func (mr *MockTaskManagerMockRecorder) CancelGlobalTask(arg0, arg1 any) *gomock.Call { +// CancelTask indicates an expected call of CancelTask. +func (mr *MockTaskManagerMockRecorder) CancelTask(arg0, arg1 any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CancelGlobalTask", reflect.TypeOf((*MockTaskManager)(nil).CancelGlobalTask), arg0, arg1) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "CancelTask", reflect.TypeOf((*MockTaskManager)(nil).CancelTask), arg0, arg1) } // CleanUpMeta mocks base method. @@ -210,41 +210,6 @@ func (mr *MockTaskManagerMockRecorder) GetAllNodes(arg0 any) *gomock.Call { return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetAllNodes", reflect.TypeOf((*MockTaskManager)(nil).GetAllNodes), arg0) } -// GetGlobalTaskByID mocks base method. -func (m *MockTaskManager) GetGlobalTaskByID(arg0 context.Context, arg1 int64) (*proto.Task, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetGlobalTaskByID", arg0, arg1) - ret0, _ := ret[0].(*proto.Task) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// GetGlobalTaskByID indicates an expected call of GetGlobalTaskByID. -func (mr *MockTaskManagerMockRecorder) GetGlobalTaskByID(arg0, arg1 any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetGlobalTaskByID", reflect.TypeOf((*MockTaskManager)(nil).GetGlobalTaskByID), arg0, arg1) -} - -// GetGlobalTasksInStates mocks base method. -func (m *MockTaskManager) GetGlobalTasksInStates(arg0 context.Context, arg1 ...any) ([]*proto.Task, error) { - m.ctrl.T.Helper() - varargs := []any{arg0} - for _, a := range arg1 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "GetGlobalTasksInStates", varargs...) - ret0, _ := ret[0].([]*proto.Task) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// GetGlobalTasksInStates indicates an expected call of GetGlobalTasksInStates. -func (mr *MockTaskManagerMockRecorder) GetGlobalTasksInStates(arg0 any, arg1 ...any) *gomock.Call { - mr.mock.ctrl.T.Helper() - varargs := append([]any{arg0}, arg1...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetGlobalTasksInStates", reflect.TypeOf((*MockTaskManager)(nil).GetGlobalTasksInStates), varargs...) -} - // GetNodesByRole mocks base method. func (m *MockTaskManager) GetNodesByRole(arg0 context.Context, arg1 string) (map[string]bool, error) { m.ctrl.T.Helper() @@ -340,6 +305,41 @@ func (mr *MockTaskManagerMockRecorder) GetSubtasksByStepAndState(arg0, arg1, arg return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSubtasksByStepAndState", reflect.TypeOf((*MockTaskManager)(nil).GetSubtasksByStepAndState), arg0, arg1, arg2, arg3) } +// GetTaskByID mocks base method. +func (m *MockTaskManager) GetTaskByID(arg0 context.Context, arg1 int64) (*proto.Task, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "GetTaskByID", arg0, arg1) + ret0, _ := ret[0].(*proto.Task) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetTaskByID indicates an expected call of GetTaskByID. +func (mr *MockTaskManagerMockRecorder) GetTaskByID(arg0, arg1 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTaskByID", reflect.TypeOf((*MockTaskManager)(nil).GetTaskByID), arg0, arg1) +} + +// GetTasksInStates mocks base method. +func (m *MockTaskManager) GetTasksInStates(arg0 context.Context, arg1 ...any) ([]*proto.Task, error) { + m.ctrl.T.Helper() + varargs := []any{arg0} + for _, a := range arg1 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetTasksInStates", varargs...) + ret0, _ := ret[0].([]*proto.Task) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// GetTasksInStates indicates an expected call of GetTasksInStates. +func (mr *MockTaskManagerMockRecorder) GetTasksInStates(arg0 any, arg1 ...any) *gomock.Call { + mr.mock.ctrl.T.Helper() + varargs := append([]any{arg0}, arg1...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTasksInStates", reflect.TypeOf((*MockTaskManager)(nil).GetTasksInStates), varargs...) +} + // PauseTask mocks base method. func (m *MockTaskManager) PauseTask(arg0 context.Context, arg1 string) (bool, error) { m.ctrl.T.Helper() @@ -397,21 +397,6 @@ func (mr *MockTaskManagerMockRecorder) TransferTasks2History(arg0, arg1 any) *go return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "TransferTasks2History", reflect.TypeOf((*MockTaskManager)(nil).TransferTasks2History), arg0, arg1) } -// UpdateGlobalTaskAndAddSubTasks mocks base method. -func (m *MockTaskManager) UpdateGlobalTaskAndAddSubTasks(arg0 context.Context, arg1 *proto.Task, arg2 []*proto.Subtask, arg3 proto.TaskState) (bool, error) { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "UpdateGlobalTaskAndAddSubTasks", arg0, arg1, arg2, arg3) - ret0, _ := ret[0].(bool) - ret1, _ := ret[1].(error) - return ret0, ret1 -} - -// UpdateGlobalTaskAndAddSubTasks indicates an expected call of UpdateGlobalTaskAndAddSubTasks. -func (mr *MockTaskManagerMockRecorder) UpdateGlobalTaskAndAddSubTasks(arg0, arg1, arg2, arg3 any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateGlobalTaskAndAddSubTasks", reflect.TypeOf((*MockTaskManager)(nil).UpdateGlobalTaskAndAddSubTasks), arg0, arg1, arg2, arg3) -} - // UpdateSubtasksSchedulerIDs mocks base method. func (m *MockTaskManager) UpdateSubtasksSchedulerIDs(arg0 context.Context, arg1 int64, arg2 []*proto.Subtask) error { m.ctrl.T.Helper() @@ -426,6 +411,21 @@ func (mr *MockTaskManagerMockRecorder) UpdateSubtasksSchedulerIDs(arg0, arg1, ar return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateSubtasksSchedulerIDs", reflect.TypeOf((*MockTaskManager)(nil).UpdateSubtasksSchedulerIDs), arg0, arg1, arg2) } +// UpdateTaskAndAddSubTasks mocks base method. +func (m *MockTaskManager) UpdateTaskAndAddSubTasks(arg0 context.Context, arg1 *proto.Task, arg2 []*proto.Subtask, arg3 proto.TaskState) (bool, error) { + m.ctrl.T.Helper() + ret := m.ctrl.Call(m, "UpdateTaskAndAddSubTasks", arg0, arg1, arg2, arg3) + ret0, _ := ret[0].(bool) + ret1, _ := ret[1].(error) + return ret0, ret1 +} + +// UpdateTaskAndAddSubTasks indicates an expected call of UpdateTaskAndAddSubTasks. +func (mr *MockTaskManagerMockRecorder) UpdateTaskAndAddSubTasks(arg0, arg1, arg2, arg3 any) *gomock.Call { + mr.mock.ctrl.T.Helper() + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "UpdateTaskAndAddSubTasks", reflect.TypeOf((*MockTaskManager)(nil).UpdateTaskAndAddSubTasks), arg0, arg1, arg2, arg3) +} + // WithNewSession mocks base method. func (m *MockTaskManager) WithNewSession(arg0 func(sessionctx.Context) error) error { m.ctrl.T.Helper() diff --git a/pkg/disttask/framework/mock/scheduler_mock.go b/pkg/disttask/framework/mock/scheduler_mock.go index 6f99256c9cede..228480f584ff1 100644 --- a/pkg/disttask/framework/mock/scheduler_mock.go +++ b/pkg/disttask/framework/mock/scheduler_mock.go @@ -74,59 +74,59 @@ func (mr *MockTaskTableMockRecorder) GetFirstSubtaskInStates(arg0, arg1, arg2, a return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetFirstSubtaskInStates", reflect.TypeOf((*MockTaskTable)(nil).GetFirstSubtaskInStates), varargs...) } -// GetGlobalTaskByID mocks base method. -func (m *MockTaskTable) GetGlobalTaskByID(arg0 context.Context, arg1 int64) (*proto.Task, error) { +// GetSubtasksInStates mocks base method. +func (m *MockTaskTable) GetSubtasksInStates(arg0 context.Context, arg1 string, arg2 int64, arg3 proto.Step, arg4 ...any) ([]*proto.Subtask, error) { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "GetGlobalTaskByID", arg0, arg1) - ret0, _ := ret[0].(*proto.Task) + varargs := []any{arg0, arg1, arg2, arg3} + for _, a := range arg4 { + varargs = append(varargs, a) + } + ret := m.ctrl.Call(m, "GetSubtasksInStates", varargs...) + ret0, _ := ret[0].([]*proto.Subtask) ret1, _ := ret[1].(error) return ret0, ret1 } -// GetGlobalTaskByID indicates an expected call of GetGlobalTaskByID. -func (mr *MockTaskTableMockRecorder) GetGlobalTaskByID(arg0, arg1 any) *gomock.Call { +// GetSubtasksInStates indicates an expected call of GetSubtasksInStates. +func (mr *MockTaskTableMockRecorder) GetSubtasksInStates(arg0, arg1, arg2, arg3 any, arg4 ...any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetGlobalTaskByID", reflect.TypeOf((*MockTaskTable)(nil).GetGlobalTaskByID), arg0, arg1) + varargs := append([]any{arg0, arg1, arg2, arg3}, arg4...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSubtasksInStates", reflect.TypeOf((*MockTaskTable)(nil).GetSubtasksInStates), varargs...) } -// GetGlobalTasksInStates mocks base method. -func (m *MockTaskTable) GetGlobalTasksInStates(arg0 context.Context, arg1 ...any) ([]*proto.Task, error) { +// GetTaskByID mocks base method. +func (m *MockTaskTable) GetTaskByID(arg0 context.Context, arg1 int64) (*proto.Task, error) { m.ctrl.T.Helper() - varargs := []any{arg0} - for _, a := range arg1 { - varargs = append(varargs, a) - } - ret := m.ctrl.Call(m, "GetGlobalTasksInStates", varargs...) - ret0, _ := ret[0].([]*proto.Task) + ret := m.ctrl.Call(m, "GetTaskByID", arg0, arg1) + ret0, _ := ret[0].(*proto.Task) ret1, _ := ret[1].(error) return ret0, ret1 } -// GetGlobalTasksInStates indicates an expected call of GetGlobalTasksInStates. -func (mr *MockTaskTableMockRecorder) GetGlobalTasksInStates(arg0 any, arg1 ...any) *gomock.Call { +// GetTaskByID indicates an expected call of GetTaskByID. +func (mr *MockTaskTableMockRecorder) GetTaskByID(arg0, arg1 any) *gomock.Call { mr.mock.ctrl.T.Helper() - varargs := append([]any{arg0}, arg1...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetGlobalTasksInStates", reflect.TypeOf((*MockTaskTable)(nil).GetGlobalTasksInStates), varargs...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTaskByID", reflect.TypeOf((*MockTaskTable)(nil).GetTaskByID), arg0, arg1) } -// GetSubtasksInStates mocks base method. -func (m *MockTaskTable) GetSubtasksInStates(arg0 context.Context, arg1 string, arg2 int64, arg3 proto.Step, arg4 ...any) ([]*proto.Subtask, error) { +// GetTasksInStates mocks base method. +func (m *MockTaskTable) GetTasksInStates(arg0 context.Context, arg1 ...any) ([]*proto.Task, error) { m.ctrl.T.Helper() - varargs := []any{arg0, arg1, arg2, arg3} - for _, a := range arg4 { + varargs := []any{arg0} + for _, a := range arg1 { varargs = append(varargs, a) } - ret := m.ctrl.Call(m, "GetSubtasksInStates", varargs...) - ret0, _ := ret[0].([]*proto.Subtask) + ret := m.ctrl.Call(m, "GetTasksInStates", varargs...) + ret0, _ := ret[0].([]*proto.Task) ret1, _ := ret[1].(error) return ret0, ret1 } -// GetSubtasksInStates indicates an expected call of GetSubtasksInStates. -func (mr *MockTaskTableMockRecorder) GetSubtasksInStates(arg0, arg1, arg2, arg3 any, arg4 ...any) *gomock.Call { +// GetTasksInStates indicates an expected call of GetTasksInStates. +func (mr *MockTaskTableMockRecorder) GetTasksInStates(arg0 any, arg1 ...any) *gomock.Call { mr.mock.ctrl.T.Helper() - varargs := append([]any{arg0, arg1, arg2, arg3}, arg4...) - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetSubtasksInStates", reflect.TypeOf((*MockTaskTable)(nil).GetSubtasksInStates), varargs...) + varargs := append([]any{arg0}, arg1...) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "GetTasksInStates", reflect.TypeOf((*MockTaskTable)(nil).GetTasksInStates), varargs...) } // HasSubtasksInStates mocks base method. diff --git a/pkg/disttask/framework/planner/planner.go b/pkg/disttask/framework/planner/planner.go index eadd73208d630..e4b0ceaa4fa16 100644 --- a/pkg/disttask/framework/planner/planner.go +++ b/pkg/disttask/framework/planner/planner.go @@ -26,7 +26,7 @@ func NewPlanner() *Planner { // Run runs the distribute plan. func (*Planner) Run(planCtx PlanCtx, plan LogicalPlan) (int64, error) { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() if err != nil { return 0, err } @@ -36,7 +36,7 @@ func (*Planner) Run(planCtx PlanCtx, plan LogicalPlan) (int64, error) { return 0, err } - return globalTaskManager.AddGlobalTaskWithSession( + return taskManager.CreateTaskWithSession( planCtx.Ctx, planCtx.SessionCtx, planCtx.TaskKey, diff --git a/pkg/disttask/framework/proto/task.go b/pkg/disttask/framework/proto/task.go index 95c797bb82bcb..529063533f32e 100644 --- a/pkg/disttask/framework/proto/task.go +++ b/pkg/disttask/framework/proto/task.go @@ -118,19 +118,28 @@ const ( StepThree Step = 3 ) -// TaskIDLabelName is the label name of task id. -const TaskIDLabelName = "task_id" +const ( + // TaskIDLabelName is the label name of task id. + TaskIDLabelName = "task_id" + // NormalPriority represents the normal priority of task. + NormalPriority = 100 +) // Task represents the task of distributed framework. +// tasks are run in the order of: priority desc, create_time asc, id asc. type Task struct { ID int64 Key string Type TaskType State TaskState Step Step + // Priority is the priority of task, the larger value means the higher priority. + // valid range is [1, 1024], default is NormalPriority. + Priority int // DispatcherID is not used now. DispatcherID string Concurrency uint64 + CreateTime time.Time StartTime time.Time StateUpdateTime time.Time Meta []byte @@ -152,9 +161,14 @@ type Subtask struct { // taken from task_key of the subtask table TaskID int64 State TaskState - // SchedulerID is the ID of scheduler, right now it's the same as instance_id, exec_id. + // Concurrency is the concurrency of the subtask, should <= task's concurrency. + // some subtasks like post-process of import into, don't consume too many resources, + // can lower this value. + Concurrency int + // ExecID is the ID of target executor, right now it's the same as instance_id, // its value is IP:PORT, see GenerateExecID - SchedulerID string + ExecID string + CreateTime time.Time // StartTime is the time when the subtask is started. // it's 0 if it hasn't started yet. StartTime time.Time @@ -167,8 +181,8 @@ type Subtask struct { } func (t *Subtask) String() string { - return fmt.Sprintf("Subtask[ID=%d, Step=%d, Type=%s, TaskID=%d, State=%s, SchedulerID=%s]", - t.ID, t.Step, t.Type, t.TaskID, t.State, t.SchedulerID) + return fmt.Sprintf("Subtask[ID=%d, Step=%d, Type=%s, TaskID=%d, State=%s, ExecID=%s]", + t.ID, t.Step, t.Type, t.TaskID, t.State, t.ExecID) } // IsFinished checks if the subtask is finished. @@ -178,12 +192,13 @@ func (t *Subtask) IsFinished() bool { } // NewSubtask create a new subtask. -func NewSubtask(step Step, taskID int64, tp TaskType, schedulerID string, meta []byte) *Subtask { +func NewSubtask(step Step, taskID int64, tp TaskType, schedulerID string, concurrency int, meta []byte) *Subtask { return &Subtask{ Step: step, Type: tp, TaskID: taskID, - SchedulerID: schedulerID, + ExecID: schedulerID, + Concurrency: concurrency, Meta: meta, } } diff --git a/pkg/disttask/framework/scheduler/interface.go b/pkg/disttask/framework/scheduler/interface.go index dc2c4aa9375af..5f3ded7867302 100644 --- a/pkg/disttask/framework/scheduler/interface.go +++ b/pkg/disttask/framework/scheduler/interface.go @@ -23,8 +23,8 @@ import ( // TaskTable defines the interface to access task table. type TaskTable interface { - GetGlobalTasksInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) - GetGlobalTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error) + GetTasksInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) + GetTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error) GetSubtasksInStates(ctx context.Context, tidbID string, taskID int64, step proto.Step, states ...interface{}) ([]*proto.Subtask, error) GetFirstSubtaskInStates(ctx context.Context, instanceID string, taskID int64, step proto.Step, states ...interface{}) (*proto.Subtask, error) diff --git a/pkg/disttask/framework/scheduler/manager.go b/pkg/disttask/framework/scheduler/manager.go index 75a494d97813e..1dba2657e5004 100644 --- a/pkg/disttask/framework/scheduler/manager.go +++ b/pkg/disttask/framework/scheduler/manager.go @@ -62,7 +62,7 @@ func (b *ManagerBuilder) setPoolFactory(poolFactory func(name string, size int32 b.newPool = poolFactory } -// Manager monitors the global task table and manages the schedulers. +// Manager monitors the task table and manages the schedulers. type Manager struct { taskTable TaskTable schedulerPool Pool @@ -138,7 +138,7 @@ func (m *Manager) Stop() { m.wg.Wait() } -// fetchAndHandleRunnableTasks fetches the runnable tasks from the global task table and handles them. +// fetchAndHandleRunnableTasks fetches the runnable tasks from the task table and handles them. func (m *Manager) fetchAndHandleRunnableTasksLoop() { defer tidbutil.Recover(metrics.LabelDomain, "fetchAndHandleRunnableTasksLoop", m.fetchAndHandleRunnableTasksLoop, false) ticker := time.NewTicker(checkTime) @@ -148,7 +148,7 @@ func (m *Manager) fetchAndHandleRunnableTasksLoop() { logutil.Logger(m.logCtx).Info("fetchAndHandleRunnableTasksLoop done") return case <-ticker.C: - tasks, err := m.taskTable.GetGlobalTasksInStates(m.ctx, proto.TaskStateRunning, proto.TaskStateReverting) + tasks, err := m.taskTable.GetTasksInStates(m.ctx, proto.TaskStateRunning, proto.TaskStateReverting) if err != nil { m.logErr(err) continue @@ -158,7 +158,7 @@ func (m *Manager) fetchAndHandleRunnableTasksLoop() { } } -// fetchAndFastCancelTasks fetches the reverting/pausing tasks from the global task table and fast cancels them. +// fetchAndFastCancelTasks fetches the reverting/pausing tasks from the task table and fast cancels them. func (m *Manager) fetchAndFastCancelTasksLoop() { defer tidbutil.Recover(metrics.LabelDomain, "fetchAndFastCancelTasksLoop", m.fetchAndFastCancelTasksLoop, false) @@ -170,7 +170,7 @@ func (m *Manager) fetchAndFastCancelTasksLoop() { logutil.Logger(m.logCtx).Info("fetchAndFastCancelTasksLoop done") return case <-ticker.C: - tasks, err := m.taskTable.GetGlobalTasksInStates(m.ctx, proto.TaskStateReverting) + tasks, err := m.taskTable.GetTasksInStates(m.ctx, proto.TaskStateReverting) if err != nil { m.logErr(err) continue @@ -178,7 +178,7 @@ func (m *Manager) fetchAndFastCancelTasksLoop() { m.onCanceledTasks(m.ctx, tasks) // cancel pending/running subtasks, and mark them as paused. - pausingTasks, err := m.taskTable.GetGlobalTasksInStates(m.ctx, proto.TaskStatePausing) + pausingTasks, err := m.taskTable.GetTasksInStates(m.ctx, proto.TaskStatePausing) if err != nil { m.logErr(err) continue @@ -361,7 +361,7 @@ func (m *Manager) onRunnableTask(task *proto.Task) { } }() }) - task, err := m.taskTable.GetGlobalTaskByID(m.ctx, task.ID) + task, err := m.taskTable.GetTaskByID(m.ctx, task.ID) if err != nil { m.logErr(err) return diff --git a/pkg/disttask/framework/scheduler/manager_test.go b/pkg/disttask/framework/scheduler/manager_test.go index ab2727b06da93..871edd85dfd4a 100644 --- a/pkg/disttask/framework/scheduler/manager_test.go +++ b/pkg/disttask/framework/scheduler/manager_test.go @@ -147,26 +147,26 @@ func TestOnRunnableTasks(t *testing.T) { mockTaskTable.EXPECT().HasSubtasksInStates(m.ctx, id, taskID, proto.StepOne, unfinishedSubtaskStates...).Return(true, nil) mockPool.EXPECT().Run(gomock.Any()).DoAndReturn(runFn) - mockTaskTable.EXPECT().GetGlobalTaskByID(m.ctx, taskID).Return(task, nil) + mockTaskTable.EXPECT().GetTaskByID(m.ctx, taskID).Return(task, nil) mockTaskTable.EXPECT().HasSubtasksInStates(m.ctx, id, taskID, proto.StepOne, unfinishedSubtaskStates...).Return(true, nil) mockInternalScheduler.EXPECT().Run(gomock.Any(), task).Return(nil) // StepTwo failed task1 := &proto.Task{ID: taskID, State: proto.TaskStateRunning, Step: proto.StepTwo} - mockTaskTable.EXPECT().GetGlobalTaskByID(m.ctx, taskID).Return(task1, nil) + mockTaskTable.EXPECT().GetTaskByID(m.ctx, taskID).Return(task1, nil) mockTaskTable.EXPECT().HasSubtasksInStates(m.ctx, id, taskID, proto.StepTwo, unfinishedSubtaskStates...).Return(true, nil) mockInternalScheduler.EXPECT().Run(gomock.Any(), task1).Return(errors.New("run err")) task2 := &proto.Task{ID: taskID, State: proto.TaskStateReverting, Step: proto.StepTwo} - mockTaskTable.EXPECT().GetGlobalTaskByID(m.ctx, taskID).Return(task2, nil) + mockTaskTable.EXPECT().GetTaskByID(m.ctx, taskID).Return(task2, nil) mockTaskTable.EXPECT().HasSubtasksInStates(m.ctx, id, taskID, proto.StepTwo, unfinishedSubtaskStates...).Return(true, nil) mockInternalScheduler.EXPECT().Rollback(gomock.Any(), task2).Return(nil) task3 := &proto.Task{ID: taskID, State: proto.TaskStateReverted, Step: proto.StepTwo} - mockTaskTable.EXPECT().GetGlobalTaskByID(m.ctx, taskID).Return(task3, nil) + mockTaskTable.EXPECT().GetTaskByID(m.ctx, taskID).Return(task3, nil) m.onRunnableTasks([]*proto.Task{task}) @@ -200,11 +200,11 @@ func TestManager(t *testing.T) { task3 := &proto.Task{ID: taskID3, State: proto.TaskStatePausing, Step: proto.StepOne, Type: "type"} mockTaskTable.EXPECT().StartManager(m.ctx, "test", "").Return(nil).Times(1) - mockTaskTable.EXPECT().GetGlobalTasksInStates(m.ctx, proto.TaskStateRunning, proto.TaskStateReverting). + mockTaskTable.EXPECT().GetTasksInStates(m.ctx, proto.TaskStateRunning, proto.TaskStateReverting). Return([]*proto.Task{task1, task2}, nil).AnyTimes() - mockTaskTable.EXPECT().GetGlobalTasksInStates(m.ctx, proto.TaskStateReverting). + mockTaskTable.EXPECT().GetTasksInStates(m.ctx, proto.TaskStateReverting). Return([]*proto.Task{task2}, nil).AnyTimes() - mockTaskTable.EXPECT().GetGlobalTasksInStates(m.ctx, proto.TaskStatePausing). + mockTaskTable.EXPECT().GetTasksInStates(m.ctx, proto.TaskStatePausing). Return([]*proto.Task{task3}, nil).AnyTimes() mockInternalScheduler.EXPECT().Init(gomock.Any()).Return(nil) // task1 @@ -213,7 +213,7 @@ func TestManager(t *testing.T) { Return(true, nil) wg, runFn := getPoolRunFn() mockPool.EXPECT().Run(gomock.Any()).DoAndReturn(runFn) - mockTaskTable.EXPECT().GetGlobalTaskByID(m.ctx, taskID1).Return(task1, nil).AnyTimes() + mockTaskTable.EXPECT().GetTaskByID(m.ctx, taskID1).Return(task1, nil).AnyTimes() mockTaskTable.EXPECT().HasSubtasksInStates(m.ctx, id, taskID1, proto.StepOne, unfinishedSubtaskStates...). Return(true, nil) @@ -228,7 +228,7 @@ func TestManager(t *testing.T) { unfinishedSubtaskStates...). Return(true, nil) mockPool.EXPECT().Run(gomock.Any()).DoAndReturn(runFn) - mockTaskTable.EXPECT().GetGlobalTaskByID(m.ctx, taskID2).Return(task2, nil).AnyTimes() + mockTaskTable.EXPECT().GetTaskByID(m.ctx, taskID2).Return(task2, nil).AnyTimes() mockTaskTable.EXPECT().HasSubtasksInStates(m.ctx, id, taskID2, proto.StepOne, unfinishedSubtaskStates...). Return(true, nil) diff --git a/pkg/disttask/framework/scheduler/scheduler.go b/pkg/disttask/framework/scheduler/scheduler.go index d0c9cbc748a9c..21933757fcec8 100644 --- a/pkg/disttask/framework/scheduler/scheduler.go +++ b/pkg/disttask/framework/scheduler/scheduler.go @@ -233,9 +233,9 @@ func (s *BaseScheduler) run(ctx context.Context, task *proto.Task) (resErr error failpoint.Inject("breakInSchedulerUT", func() { failpoint.Break() }) - newTask, err := s.taskTable.GetGlobalTaskByID(runCtx, task.ID) + newTask, err := s.taskTable.GetTaskByID(runCtx, task.ID) if err != nil { - logutil.Logger(s.logCtx).Warn("GetGlobalTaskByID meets error", zap.Error(err)) + logutil.Logger(s.logCtx).Warn("GetTaskByID meets error", zap.Error(err)) continue } // When the task move to next step or task state changes, the scheduler should exit. @@ -349,9 +349,9 @@ func (s *BaseScheduler) runSubtask(ctx context.Context, executor execute.Subtask if err != nil { logutil.BgLogger().Error("get task manager failed", zap.Error(err)) } else { - err = mgr.CancelGlobalTask(ctx, int64(taskID)) + err = mgr.CancelTask(ctx, int64(taskID)) if err != nil { - logutil.BgLogger().Error("cancel global task failed", zap.Error(err)) + logutil.BgLogger().Error("cancel task failed", zap.Error(err)) } } } @@ -574,7 +574,7 @@ func (s *BaseScheduler) finishSubtask(ctx context.Context, subtask *proto.Subtas backoffer := backoff.NewExponential(dispatcher.RetrySQLInterval, 2, dispatcher.RetrySQLMaxInterval) err := handle.RunWithRetry(ctx, dispatcher.RetrySQLTimes, backoffer, logger, func(ctx context.Context) (bool, error) { - return true, s.taskTable.FinishSubtask(ctx, subtask.SchedulerID, subtask.ID, subtask.Meta) + return true, s.taskTable.FinishSubtask(ctx, subtask.ExecID, subtask.ID, subtask.Meta) }, ) if err != nil { @@ -585,7 +585,7 @@ func (s *BaseScheduler) finishSubtask(ctx context.Context, subtask *proto.Subtas func (s *BaseScheduler) updateSubtaskStateAndError(ctx context.Context, subtask *proto.Subtask, state proto.TaskState, subTaskErr error) { metrics.DecDistTaskSubTaskCnt(subtask) metrics.EndDistTaskSubTask(subtask) - s.updateSubtaskStateAndErrorImpl(ctx, subtask.SchedulerID, subtask.ID, state, subTaskErr) + s.updateSubtaskStateAndErrorImpl(ctx, subtask.ExecID, subtask.ID, state, subTaskErr) subtask.State = state metrics.IncDistTaskSubTaskCnt(subtask) if !subtask.IsFinished() { diff --git a/pkg/disttask/framework/scheduler/scheduler_test.go b/pkg/disttask/framework/scheduler/scheduler_test.go index 508cdcfae1a13..ed52ef2db40d0 100644 --- a/pkg/disttask/framework/scheduler/scheduler_test.go +++ b/pkg/disttask/framework/scheduler/scheduler_test.go @@ -80,10 +80,10 @@ func TestSchedulerRun(t *testing.T) { mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(runSubtaskErr) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(gomock.Any(), "id", taskID, proto.TaskStateFailed, gomock.Any()).Return(nil) @@ -96,17 +96,17 @@ func TestSchedulerRun(t *testing.T) { mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(nil) mockSubtaskExecutor.EXPECT().OnFinished(gomock.Any(), gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().FinishSubtask(gomock.Any(), "id", int64(1), gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) - mockSubtaskTable.EXPECT().GetGlobalTaskByID(gomock.Any(), gomock.Any()).Return(&proto.Task{ID: taskID, Step: proto.StepTwo}, nil) + mockSubtaskTable.EXPECT().GetTaskByID(gomock.Any(), gomock.Any()).Return(&proto.Task{ID: taskID, Step: proto.StepTwo}, nil) mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil) err = scheduler.Run(runCtx, task) require.NoError(t, err) @@ -116,13 +116,13 @@ func TestSchedulerRun(t *testing.T) { mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return( []*proto.Subtask{ - {ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, - {ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, + {ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, + {ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, }, nil) // first round of the run loop mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), int64(1)).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(nil) mockSubtaskExecutor.EXPECT().OnFinished(gomock.Any(), gomock.Any()).Return(nil) @@ -130,7 +130,7 @@ func TestSchedulerRun(t *testing.T) { // second round of the run loop mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), int64(2)).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(nil) mockSubtaskExecutor.EXPECT().OnFinished(gomock.Any(), gomock.Any()).Return(nil) @@ -138,7 +138,7 @@ func TestSchedulerRun(t *testing.T) { // third round of the run loop mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) - mockSubtaskTable.EXPECT().GetGlobalTaskByID(gomock.Any(), gomock.Any()).Return(&proto.Task{ID: taskID, Step: proto.StepTwo}, nil) + mockSubtaskTable.EXPECT().GetTaskByID(gomock.Any(), gomock.Any()).Return(&proto.Task{ID: taskID, Step: proto.StepTwo}, nil) mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil) err = scheduler.Run(runCtx, task) require.NoError(t, err) @@ -146,7 +146,7 @@ func TestSchedulerRun(t *testing.T) { // run previous left subtask in running state again, but the subtask is not // idempotent, so fail it. subtaskID := int64(2) - theSubtask := &proto.Subtask{ID: subtaskID, Type: tp, Step: proto.StepOne, State: proto.TaskStateRunning, SchedulerID: "id"} + theSubtask := &proto.Subtask{ID: subtaskID, Type: tp, Step: proto.StepOne, State: proto.TaskStateRunning, ExecID: "id"} mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{theSubtask}, nil) mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) @@ -160,7 +160,7 @@ func TestSchedulerRun(t *testing.T) { // run previous left subtask in running state again, but the subtask idempotent, // run it again. - theSubtask = &proto.Subtask{ID: subtaskID, Type: tp, Step: proto.StepOne, State: proto.TaskStateRunning, SchedulerID: "id"} + theSubtask = &proto.Subtask{ID: subtaskID, Type: tp, Step: proto.StepOne, State: proto.TaskStateRunning, ExecID: "id"} mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{theSubtask}, nil) mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) @@ -174,7 +174,7 @@ func TestSchedulerRun(t *testing.T) { // second round of the run loop mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) - mockSubtaskTable.EXPECT().GetGlobalTaskByID(gomock.Any(), gomock.Any()).Return(&proto.Task{ID: taskID, Step: proto.StepTwo}, nil) + mockSubtaskTable.EXPECT().GetTaskByID(gomock.Any(), gomock.Any()).Return(&proto.Task{ID: taskID, Step: proto.StepTwo}, nil) mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil) err = scheduler.Run(runCtx, task) require.NoError(t, err) @@ -182,11 +182,11 @@ func TestSchedulerRun(t *testing.T) { // 6. cancel mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(ErrCancelSubtask) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(gomock.Any(), "id", taskID, proto.TaskStateCanceled, gomock.Any()).Return(nil) @@ -197,11 +197,11 @@ func TestSchedulerRun(t *testing.T) { // 7. RunSubtask return context.Canceled mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(context.Canceled) mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil) @@ -211,11 +211,11 @@ func TestSchedulerRun(t *testing.T) { // 8. grpc cancel mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) grpcErr := status.Error(codes.Canceled, "test cancel") mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(grpcErr) @@ -226,11 +226,11 @@ func TestSchedulerRun(t *testing.T) { // 9. annotate grpc cancel mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 2, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) grpcErr = status.Error(codes.Canceled, "test cancel") annotatedError := errors.Annotatef( @@ -293,7 +293,7 @@ func TestSchedulerRollback(t *testing.T) { mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, - unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 1, State: proto.TaskStateRevertPending, SchedulerID: "id"}, nil) + unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 1, State: proto.TaskStateRevertPending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", taskID, proto.TaskStateReverting, nil).Return(nil) mockSubtaskExecutor.EXPECT().Rollback(gomock.Any()).Return(rollbackErr) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", taskID, proto.TaskStateRevertFailed, nil).Return(nil) @@ -302,15 +302,15 @@ func TestSchedulerRollback(t *testing.T) { // 5. rollback success mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, - unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ID: 1, SchedulerID: "id"}, nil) + unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ID: 1, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", int64(1), proto.TaskStateCanceled, nil).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, - unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ID: 2, SchedulerID: "id"}, nil) + unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ID: 2, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", int64(2), proto.TaskStateCanceled, nil).Return(nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, - unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 3, State: proto.TaskStateRevertPending, SchedulerID: "id"}, nil) + unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 3, State: proto.TaskStateRevertPending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", int64(3), proto.TaskStateReverting, nil).Return(nil) mockSubtaskExecutor.EXPECT().Rollback(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", int64(3), proto.TaskStateReverted, nil).Return(nil) @@ -321,7 +321,7 @@ func TestSchedulerRollback(t *testing.T) { mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, - unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 3, State: proto.TaskStateReverting, SchedulerID: "id"}, nil) + unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 3, State: proto.TaskStateReverting, ExecID: "id"}, nil) mockSubtaskExecutor.EXPECT().Rollback(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", int64(3), proto.TaskStateReverted, nil).Return(nil) err = scheduler.Rollback(runCtx, &proto.Task{Step: proto.StepOne, Type: tp, ID: taskID}) @@ -377,10 +377,10 @@ func TestScheduler(t *testing.T) { mockSubtaskExecutor.EXPECT().Init(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().GetSubtasksInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return([]*proto.Subtask{{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}}, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(gomock.Any(), "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(&proto.Subtask{ - ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, SchedulerID: "id"}, nil) + ID: 1, Type: tp, Step: proto.StepOne, State: proto.TaskStatePending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().StartSubtask(gomock.Any(), taskID).Return(nil) mockSubtaskExecutor.EXPECT().RunSubtask(gomock.Any(), gomock.Any()).Return(runSubtaskErr) mockSubtaskExecutor.EXPECT().Cleanup(gomock.Any()).Return(nil) @@ -391,7 +391,7 @@ func TestScheduler(t *testing.T) { mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, unfinishedNormalSubtaskStates...).Return(nil, nil) mockSubtaskTable.EXPECT().GetFirstSubtaskInStates(runCtx, "id", taskID, proto.StepOne, - unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 1, Type: tp, State: proto.TaskStateRevertPending, SchedulerID: "id"}, nil) + unfinishedRevertSubtaskStates...).Return(&proto.Subtask{ID: 1, Type: tp, State: proto.TaskStateRevertPending, ExecID: "id"}, nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", taskID, proto.TaskStateReverting, nil).Return(nil) mockSubtaskExecutor.EXPECT().Rollback(gomock.Any()).Return(nil) mockSubtaskTable.EXPECT().UpdateSubtaskStateAndError(runCtx, "id", taskID, proto.TaskStateReverted, nil).Return(nil) diff --git a/pkg/disttask/framework/scheduler/scheduler_testkit_test.go b/pkg/disttask/framework/scheduler/scheduler_testkit_test.go index 2b47b79c544f6..3f33c3096ee26 100644 --- a/pkg/disttask/framework/scheduler/scheduler_testkit_test.go +++ b/pkg/disttask/framework/scheduler/scheduler_testkit_test.go @@ -35,19 +35,19 @@ import ( ) func runOneTask(ctx context.Context, t *testing.T, mgr *storage.TaskManager, taskKey string, subtaskCnt int) { - taskID, err := mgr.AddNewGlobalTask(ctx, taskKey, proto.TaskTypeExample, 1, nil) + taskID, err := mgr.CreateTask(ctx, taskKey, proto.TaskTypeExample, 1, nil) require.NoError(t, err) - task, err := mgr.GetGlobalTaskByID(ctx, taskID) + task, err := mgr.GetTaskByID(ctx, taskID) require.NoError(t, err) // 1. stepOne task.Step = proto.StepOne task.State = proto.TaskStateRunning - _, err = mgr.UpdateGlobalTaskAndAddSubTasks(ctx, task, nil, proto.TaskStatePending) + _, err = mgr.UpdateTaskAndAddSubTasks(ctx, task, nil, proto.TaskStatePending) require.NoError(t, err) for i := 0; i < subtaskCnt; i++ { - require.NoError(t, mgr.AddNewSubTask(ctx, taskID, proto.StepOne, "test", nil, proto.TaskTypeExample, false)) + require.NoError(t, mgr.CreateSubTask(ctx, taskID, proto.StepOne, "test", nil, proto.TaskTypeExample, false)) } - task, err = mgr.GetGlobalTaskByID(ctx, taskID) + task, err = mgr.GetTaskByID(ctx, taskID) require.NoError(t, err) factory := scheduler.GetSchedulerFactory(task.Type) require.NotNil(t, factory) @@ -55,12 +55,12 @@ func runOneTask(ctx context.Context, t *testing.T, mgr *storage.TaskManager, tas require.NoError(t, scheduler.Run(ctx, task)) // 2. stepTwo task.Step = proto.StepTwo - _, err = mgr.UpdateGlobalTaskAndAddSubTasks(ctx, task, nil, proto.TaskStateRunning) + _, err = mgr.UpdateTaskAndAddSubTasks(ctx, task, nil, proto.TaskStateRunning) require.NoError(t, err) for i := 0; i < subtaskCnt; i++ { - require.NoError(t, mgr.AddNewSubTask(ctx, taskID, proto.StepTwo, "test", nil, proto.TaskTypeExample, false)) + require.NoError(t, mgr.CreateSubTask(ctx, taskID, proto.StepTwo, "test", nil, proto.TaskTypeExample, false)) } - task, err = mgr.GetGlobalTaskByID(ctx, taskID) + task, err = mgr.GetTaskByID(ctx, taskID) require.NoError(t, err) require.NoError(t, scheduler.Run(ctx, task)) } diff --git a/pkg/disttask/framework/storage/table_test.go b/pkg/disttask/framework/storage/table_test.go index 71a2aad01cc77..d7d6eb4658fbc 100644 --- a/pkg/disttask/framework/storage/table_test.go +++ b/pkg/disttask/framework/storage/table_test.go @@ -59,36 +59,41 @@ func GetTaskManager(t *testing.T, pool *pools.ResourcePool) *storage.TaskManager return manager } -func TestGlobalTaskTable(t *testing.T) { +func TestTaskTable(t *testing.T) { pool := GetResourcePool(t) gm := GetTaskManager(t, pool) defer pool.Close() ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "table_test") - id, err := gm.AddNewGlobalTask(ctx, "key1", "test", 4, []byte("test")) + timeBeforeCreate := time.Unix(time.Now().Unix(), 0) + id, err := gm.CreateTask(ctx, "key1", "test", 4, []byte("test")) require.NoError(t, err) require.Equal(t, int64(1), id) - task, err := gm.GetNewGlobalTask(ctx) + task, err := gm.GetOneTask(ctx) require.NoError(t, err) require.Equal(t, int64(1), task.ID) require.Equal(t, "key1", task.Key) require.Equal(t, proto.TaskType("test"), task.Type) require.Equal(t, proto.TaskStatePending, task.State) + require.Equal(t, proto.NormalPriority, task.Priority) require.Equal(t, uint64(4), task.Concurrency) + require.Equal(t, proto.StepInit, task.Step) require.Equal(t, []byte("test"), task.Meta) + require.GreaterOrEqual(t, task.CreateTime, timeBeforeCreate) + require.Nil(t, task.Error) - task2, err := gm.GetGlobalTaskByID(ctx, 1) + task2, err := gm.GetTaskByID(ctx, 1) require.NoError(t, err) require.Equal(t, task, task2) - task3, err := gm.GetGlobalTasksInStates(ctx, proto.TaskStatePending) + task3, err := gm.GetTasksInStates(ctx, proto.TaskStatePending) require.NoError(t, err) require.Len(t, task3, 1) require.Equal(t, task, task3[0]) - task4, err := gm.GetGlobalTasksInStates(ctx, proto.TaskStatePending, proto.TaskStateRunning) + task4, err := gm.GetTasksInStates(ctx, proto.TaskStatePending, proto.TaskStateRunning) require.NoError(t, err) require.Len(t, task4, 1) require.Equal(t, task, task4[0]) @@ -96,34 +101,34 @@ func TestGlobalTaskTable(t *testing.T) { prevState := task.State task.State = proto.TaskStateRunning - retryable, err := gm.UpdateGlobalTaskAndAddSubTasks(ctx, task, nil, prevState) + retryable, err := gm.UpdateTaskAndAddSubTasks(ctx, task, nil, prevState) require.NoError(t, err) require.Equal(t, true, retryable) - task5, err := gm.GetGlobalTasksInStates(ctx, proto.TaskStateRunning) + task5, err := gm.GetTasksInStates(ctx, proto.TaskStateRunning) require.NoError(t, err) require.Len(t, task5, 1) require.Equal(t, task.State, task5[0].State) - task6, err := gm.GetGlobalTaskByKey(ctx, "key1") + task6, err := gm.GetTaskByKey(ctx, "key1") require.NoError(t, err) require.Len(t, task5, 1) require.Equal(t, task.State, task6.State) // test cannot insert task with dup key - _, err = gm.AddNewGlobalTask(ctx, "key1", "test2", 4, []byte("test2")) + _, err = gm.CreateTask(ctx, "key1", "test2", 4, []byte("test2")) require.EqualError(t, err, "[kv:1062]Duplicate entry 'key1' for key 'tidb_global_task.task_key'") - // test cancel global task - id, err = gm.AddNewGlobalTask(ctx, "key2", "test", 4, []byte("test")) + // test cancel task + id, err = gm.CreateTask(ctx, "key2", "test", 4, []byte("test")) require.NoError(t, err) - cancelling, err := gm.IsGlobalTaskCancelling(ctx, id) + cancelling, err := gm.IsTaskCancelling(ctx, id) require.NoError(t, err) require.False(t, cancelling) - require.NoError(t, gm.CancelGlobalTask(ctx, id)) - cancelling, err = gm.IsGlobalTaskCancelling(ctx, id) + require.NoError(t, gm.CancelTask(ctx, id)) + cancelling, err = gm.IsTaskCancelling(ctx, id) require.NoError(t, err) require.True(t, cancelling) } @@ -135,7 +140,26 @@ func TestSubTaskTable(t *testing.T) { ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "table_test") - err := sm.AddNewSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false) + timeBeforeCreate := time.Unix(time.Now().Unix(), 0) + id, err := sm.CreateTask(ctx, "key1", "test", 4, []byte("test")) + require.NoError(t, err) + require.Equal(t, int64(1), id) + _, err = sm.UpdateTaskAndAddSubTasks( + ctx, + &proto.Task{ + ID: 1, + State: proto.TaskStateRunning, + }, + []*proto.Subtask{ + { + Step: proto.StepInit, + Type: proto.TaskTypeExample, + Concurrency: 11, + ExecID: "tidb1", + Meta: []byte("test"), + }, + }, proto.TaskStatePending, + ) require.NoError(t, err) nilTask, err := sm.GetFirstSubtaskInStates(ctx, "tidb2", 1, proto.StepInit, proto.TaskStatePending) @@ -144,13 +168,18 @@ func TestSubTaskTable(t *testing.T) { subtask, err := sm.GetFirstSubtaskInStates(ctx, "tidb1", 1, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) + require.Equal(t, proto.StepInit, subtask.Step) + require.Equal(t, int64(1), subtask.TaskID) require.Equal(t, proto.TaskTypeExample, subtask.Type) require.Equal(t, int64(1), subtask.TaskID) require.Equal(t, proto.TaskStatePending, subtask.State) - require.Equal(t, "tidb1", subtask.SchedulerID) + require.Equal(t, "tidb1", subtask.ExecID) require.Equal(t, []byte("test"), subtask.Meta) + require.Equal(t, 11, subtask.Concurrency) + require.GreaterOrEqual(t, subtask.CreateTime, timeBeforeCreate) require.Zero(t, subtask.StartTime) require.Zero(t, subtask.UpdateTime) + require.Equal(t, "{}", subtask.Summary) subtask2, err := sm.GetFirstSubtaskInStates(ctx, "tidb1", 1, proto.StepInit, proto.TaskStatePending, proto.TaskStateReverted) require.NoError(t, err) @@ -190,7 +219,7 @@ func TestSubTaskTable(t *testing.T) { require.Equal(t, proto.TaskTypeExample, subtask.Type) require.Equal(t, int64(1), subtask.TaskID) require.Equal(t, proto.TaskStateRunning, subtask.State) - require.Equal(t, "tidb1", subtask.SchedulerID) + require.Equal(t, "tidb1", subtask.ExecID) require.Equal(t, []byte("test"), subtask.Meta) require.GreaterOrEqual(t, subtask.StartTime, ts) require.GreaterOrEqual(t, subtask.UpdateTime, ts) @@ -218,7 +247,7 @@ func TestSubTaskTable(t *testing.T) { require.NoError(t, err) require.False(t, ok) - err = sm.AddNewSubTask(ctx, 2, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, true) + err = sm.CreateSubTask(ctx, 2, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, true) require.NoError(t, err) cnt, err = sm.GetSubtaskInStatesCnt(ctx, 2, proto.TaskStateRevertPending) @@ -246,7 +275,7 @@ func TestSubTaskTable(t *testing.T) { require.Equal(t, int64(100), rowCount) // test UpdateErrorToSubtask do update start/update time - err = sm.AddNewSubTask(ctx, 3, proto.StepInit, "for_test", []byte("test"), proto.TaskTypeExample, false) + err = sm.CreateSubTask(ctx, 3, proto.StepInit, "for_test", []byte("test"), proto.TaskTypeExample, false) require.NoError(t, err) require.NoError(t, sm.UpdateErrorToSubtask(ctx, "for_test", 3, errors.New("fail"))) subtask, err = sm.GetFirstSubtaskInStates(ctx, "for_test", 3, proto.StepInit, proto.TaskStateFailed) @@ -256,7 +285,7 @@ func TestSubTaskTable(t *testing.T) { require.Greater(t, subtask.UpdateTime, ts) // test FinishSubtask do update update time - err = sm.AddNewSubTask(ctx, 4, proto.StepInit, "for_test1", []byte("test"), proto.TaskTypeExample, false) + err = sm.CreateSubTask(ctx, 4, proto.StepInit, "for_test1", []byte("test"), proto.TaskTypeExample, false) require.NoError(t, err) subtask, err = sm.GetFirstSubtaskInStates(ctx, "for_test1", 4, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) @@ -285,39 +314,39 @@ func TestSubTaskTable(t *testing.T) { // test UpdateSubtasksSchedulerIDs // 1. update one subtask - require.NoError(t, sm.AddNewSubTask(ctx, 5, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 5, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 5, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) - subtasks[0].SchedulerID = "tidb2" + subtasks[0].ExecID = "tidb2" require.NoError(t, sm.UpdateSubtasksSchedulerIDs(ctx, 5, subtasks)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 5, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) - require.Equal(t, "tidb2", subtasks[0].SchedulerID) + require.Equal(t, "tidb2", subtasks[0].ExecID) // 2. update 2 subtasks - require.NoError(t, sm.AddNewSubTask(ctx, 5, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 5, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 5, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) - subtasks[0].SchedulerID = "tidb3" + subtasks[0].ExecID = "tidb3" require.NoError(t, sm.UpdateSubtasksSchedulerIDs(ctx, 5, subtasks)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 5, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) - require.Equal(t, "tidb3", subtasks[0].SchedulerID) - require.Equal(t, "tidb1", subtasks[1].SchedulerID) + require.Equal(t, "tidb3", subtasks[0].ExecID) + require.Equal(t, "tidb1", subtasks[1].ExecID) // update fail require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, "tidb1", subtasks[0].ID, proto.TaskStateRunning, nil)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 5, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) - require.Equal(t, "tidb3", subtasks[0].SchedulerID) - subtasks[0].SchedulerID = "tidb2" + require.Equal(t, "tidb3", subtasks[0].ExecID) + subtasks[0].ExecID = "tidb2" // update success require.NoError(t, sm.UpdateSubtasksSchedulerIDs(ctx, 5, subtasks)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 5, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) - require.Equal(t, "tidb2", subtasks[0].SchedulerID) + require.Equal(t, "tidb2", subtasks[0].ExecID) // test GetSubtasksByExecIdsAndStepAndState - require.NoError(t, sm.AddNewSubTask(ctx, 6, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) - require.NoError(t, sm.AddNewSubTask(ctx, 6, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 6, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 6, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) subtasks, err = sm.GetSubtasksByStepAndState(ctx, 6, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, "tidb1", subtasks[0].ID, proto.TaskStateRunning, nil)) @@ -327,7 +356,7 @@ func TestSubTaskTable(t *testing.T) { subtasks, err = sm.GetSubtasksByExecIdsAndStepAndState(ctx, []string{"tidb1"}, 6, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) require.Equal(t, 1, len(subtasks)) - require.NoError(t, sm.AddNewSubTask(ctx, 6, proto.StepInit, "tidb2", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 6, proto.StepInit, "tidb2", []byte("test"), proto.TaskTypeExample, false)) subtasks, err = sm.GetSubtasksByExecIdsAndStepAndState(ctx, []string{"tidb1", "tidb2"}, 6, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) require.Equal(t, 2, len(subtasks)) @@ -336,18 +365,18 @@ func TestSubTaskTable(t *testing.T) { require.Equal(t, 1, len(subtasks)) } -func TestBothGlobalAndSubTaskTable(t *testing.T) { +func TestBothTaskAndSubTaskTable(t *testing.T) { pool := GetResourcePool(t) sm := GetTaskManager(t, pool) defer pool.Close() ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "table_test") - id, err := sm.AddNewGlobalTask(ctx, "key1", "test", 4, []byte("test")) + id, err := sm.CreateTask(ctx, "key1", "test", 4, []byte("test")) require.NoError(t, err) require.Equal(t, int64(1), id) - task, err := sm.GetNewGlobalTask(ctx) + task, err := sm.GetOneTask(ctx) require.NoError(t, err) require.Equal(t, proto.TaskStatePending, task.State) @@ -356,23 +385,23 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { task.State = proto.TaskStateRunning subTasks := []*proto.Subtask{ { - Step: proto.StepInit, - Type: proto.TaskTypeExample, - SchedulerID: "instance1", - Meta: []byte("m1"), + Step: proto.StepInit, + Type: proto.TaskTypeExample, + ExecID: "instance1", + Meta: []byte("m1"), }, { - Step: proto.StepInit, - Type: proto.TaskTypeExample, - SchedulerID: "instance2", - Meta: []byte("m2"), + Step: proto.StepInit, + Type: proto.TaskTypeExample, + ExecID: "instance2", + Meta: []byte("m2"), }, } - retryable, err := sm.UpdateGlobalTaskAndAddSubTasks(ctx, task, subTasks, prevState) + retryable, err := sm.UpdateTaskAndAddSubTasks(ctx, task, subTasks, prevState) require.NoError(t, err) require.Equal(t, true, retryable) - task, err = sm.GetGlobalTaskByID(ctx, 1) + task, err = sm.GetTaskByID(ctx, 1) require.NoError(t, err) require.Equal(t, proto.TaskStateRunning, task.State) @@ -397,23 +426,23 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { task.State = proto.TaskStateReverting subTasks = []*proto.Subtask{ { - Step: proto.StepInit, - Type: proto.TaskTypeExample, - SchedulerID: "instance3", - Meta: []byte("m3"), + Step: proto.StepInit, + Type: proto.TaskTypeExample, + ExecID: "instance3", + Meta: []byte("m3"), }, { - Step: proto.StepInit, - Type: proto.TaskTypeExample, - SchedulerID: "instance4", - Meta: []byte("m4"), + Step: proto.StepInit, + Type: proto.TaskTypeExample, + ExecID: "instance4", + Meta: []byte("m4"), }, } - retryable, err = sm.UpdateGlobalTaskAndAddSubTasks(ctx, task, subTasks, prevState) + retryable, err = sm.UpdateTaskAndAddSubTasks(ctx, task, subTasks, prevState) require.NoError(t, err) require.Equal(t, true, retryable) - task, err = sm.GetGlobalTaskByID(ctx, 1) + task, err = sm.GetTaskByID(ctx, 1) require.NoError(t, err) require.Equal(t, proto.TaskStateReverting, task.State) @@ -441,11 +470,11 @@ func TestBothGlobalAndSubTaskTable(t *testing.T) { }() prevState = task.State task.State = proto.TaskStateFailed - retryable, err = sm.UpdateGlobalTaskAndAddSubTasks(ctx, task, subTasks, prevState) + retryable, err = sm.UpdateTaskAndAddSubTasks(ctx, task, subTasks, prevState) require.EqualError(t, err, "updateTaskErr") require.Equal(t, true, retryable) - task, err = sm.GetGlobalTaskByID(ctx, 1) + task, err = sm.GetTaskByID(ctx, 1) require.NoError(t, err) require.Equal(t, proto.TaskStateReverting, task.State) @@ -517,11 +546,11 @@ func TestSubtaskHistoryTable(t *testing.T) { finishedMeta = "finished" ) - require.NoError(t, sm.AddNewSubTask(ctx, taskID, proto.StepInit, tidb1, []byte(meta), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, taskID, proto.StepInit, tidb1, []byte(meta), proto.TaskTypeExample, false)) require.NoError(t, sm.FinishSubtask(ctx, tidb1, subTask1, []byte(finishedMeta))) - require.NoError(t, sm.AddNewSubTask(ctx, taskID, proto.StepInit, tidb2, []byte(meta), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, taskID, proto.StepInit, tidb2, []byte(meta), proto.TaskTypeExample, false)) require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, tidb2, subTask2, proto.TaskStateCanceled, nil)) - require.NoError(t, sm.AddNewSubTask(ctx, taskID, proto.StepInit, tidb3, []byte(meta), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, taskID, proto.StepInit, tidb3, []byte(meta), proto.TaskTypeExample, false)) require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, tidb3, subTask3, proto.TaskStateFailed, nil)) subTasks, err := storage.GetSubtasksByTaskIDForTest(ctx, sm, taskID) @@ -554,7 +583,7 @@ func TestSubtaskHistoryTable(t *testing.T) { }() time.Sleep(2 * time.Second) - require.NoError(t, sm.AddNewSubTask(ctx, taskID2, proto.StepInit, tidb1, []byte(meta), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, taskID2, proto.StepInit, tidb1, []byte(meta), proto.TaskTypeExample, false)) require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, tidb1, subTask4, proto.TaskStateFailed, nil)) require.NoError(t, sm.TransferSubTasks2History(ctx, taskID2)) @@ -572,18 +601,18 @@ func TestTaskHistoryTable(t *testing.T) { ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "table_test") - _, err := gm.AddNewGlobalTask(ctx, "1", proto.TaskTypeExample, 1, nil) + _, err := gm.CreateTask(ctx, "1", proto.TaskTypeExample, 1, nil) require.NoError(t, err) - taskID, err := gm.AddNewGlobalTask(ctx, "2", proto.TaskTypeExample, 1, nil) + taskID, err := gm.CreateTask(ctx, "2", proto.TaskTypeExample, 1, nil) require.NoError(t, err) - tasks, err := gm.GetGlobalTasksInStates(ctx, proto.TaskStatePending) + tasks, err := gm.GetTasksInStates(ctx, proto.TaskStatePending) require.NoError(t, err) require.Equal(t, 2, len(tasks)) require.NoError(t, gm.TransferTasks2History(ctx, tasks)) - tasks, err = gm.GetGlobalTasksInStates(ctx, proto.TaskStatePending) + tasks, err = gm.GetTasksInStates(ctx, proto.TaskStatePending) require.NoError(t, err) require.Equal(t, 0, len(tasks)) num, err := storage.GetTasksFromHistoryForTest(ctx, gm) @@ -594,14 +623,14 @@ func TestTaskHistoryTable(t *testing.T) { require.NoError(t, err) require.NotNil(t, task) - task, err = gm.GetGlobalTaskByKeyWithHistory(ctx, "1") + task, err = gm.GetTaskByKeyWithHistory(ctx, "1") require.NoError(t, err) require.NotNil(t, task) // task with fail transfer - _, err = gm.AddNewGlobalTask(ctx, "3", proto.TaskTypeExample, 1, nil) + _, err = gm.CreateTask(ctx, "3", proto.TaskTypeExample, 1, nil) require.NoError(t, err) - tasks, err = gm.GetGlobalTasksInStates(ctx, proto.TaskStatePending) + tasks, err = gm.GetTasksInStates(ctx, proto.TaskStatePending) require.NoError(t, err) require.Equal(t, 1, len(tasks)) tasks[0].Error = errors.New("mock err") @@ -618,9 +647,9 @@ func TestPauseAndResume(t *testing.T) { ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "table_test") - require.NoError(t, sm.AddNewSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) - require.NoError(t, sm.AddNewSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) - require.NoError(t, sm.AddNewSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) // 1.1 pause all subtasks. require.NoError(t, sm.PauseSubtasks(ctx, "tidb1", 1)) cnt, err := sm.GetSubtaskInStatesCnt(ctx, 1, proto.TaskStatePaused) @@ -652,7 +681,7 @@ func TestCancelAndExecIdChanged(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) ctx = util.WithInternalSourceType(ctx, "table_test") - require.NoError(t, sm.AddNewSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) + require.NoError(t, sm.CreateSubTask(ctx, 1, proto.StepInit, "tidb1", []byte("test"), proto.TaskTypeExample, false)) subtask, err := sm.GetFirstSubtaskInStates(ctx, "tidb1", 1, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) // 1. cancel the ctx, then update subtask state. @@ -669,7 +698,7 @@ func TestCancelAndExecIdChanged(t *testing.T) { // exec_id changed require.NoError(t, sm.UpdateSubtaskExecID(ctx, "tidb2", subtask.ID)) // exec_id in memory unchanged, call UpdateSubtaskStateAndError. - require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, subtask.SchedulerID, subtask.ID, proto.TaskStateFailed, nil)) + require.NoError(t, sm.UpdateSubtaskStateAndError(ctx, subtask.ExecID, subtask.ID, proto.TaskStateFailed, nil)) subtask, err = sm.GetFirstSubtaskInStates(ctx, "tidb2", 1, proto.StepInit, proto.TaskStatePending) require.NoError(t, err) // state unchanged diff --git a/pkg/disttask/framework/storage/task_table.go b/pkg/disttask/framework/storage/task_table.go index 40bcc537c7722..cce103a6d82d4 100644 --- a/pkg/disttask/framework/storage/task_table.go +++ b/pkg/disttask/framework/storage/task_table.go @@ -38,7 +38,16 @@ import ( "go.uber.org/zap" ) -const defaultSubtaskKeepDays = 14 +const ( + defaultSubtaskKeepDays = 14 + + taskColumns = `id, task_key, type, dispatcher_id, state, start_time, state_update_time, + meta, concurrency, step, error, priority, create_time` + subtaskColumns = `id, step, task_key, type, exec_id, state, concurrency, create_time, + start_time, state_update_time, meta, summary` + insertSubtaskBasic = `insert into mysql.tidb_background_subtask( + step, task_key, exec_id, meta, state, type, concurrency, create_time, checkpoint, summary) values ` +) // SessionExecutor defines the interface for executing SQLs in a session. type SessionExecutor interface { @@ -48,7 +57,7 @@ type SessionExecutor interface { WithNewTxn(ctx context.Context, fn func(se sessionctx.Context) error) error } -// TaskManager is the manager of global/sub task. +// TaskManager is the manager of task and subtask. type TaskManager struct { sePool sessionPool } @@ -78,7 +87,7 @@ func NewTaskManager(sePool sessionPool) *TaskManager { func GetTaskManager() (*TaskManager, error) { v := taskManagerInstance.Load() if v == nil { - return nil, errors.New("global task manager is not initialized") + return nil, errors.New("task manager is not initialized") } return v, nil } @@ -102,8 +111,8 @@ func ExecSQL(ctx context.Context, se sessionctx.Context, sql string, args ...int return nil, nil } -// row2GlobeTask converts a row to a global task. -func row2GlobeTask(r chunk.Row) *proto.Task { +// row2Task converts a row to a task. +func row2Task(r chunk.Row) *proto.Task { task := &proto.Task{ ID: r.GetInt64(0), Key: r.GetString(1), @@ -113,6 +122,7 @@ func row2GlobeTask(r chunk.Row) *proto.Task { Meta: r.GetBytes(7), Concurrency: uint64(r.GetInt64(8)), Step: proto.Step(r.GetInt64(9)), + Priority: int(r.GetInt64(11)), } if !r.IsNull(10) { errBytes := r.GetBytes(10) @@ -125,9 +135,9 @@ func row2GlobeTask(r chunk.Row) *proto.Task { task.Error = stdErr } } - // TODO: convert to local time. - task.StartTime, _ = r.GetTime(5).GoTime(time.UTC) - task.StateUpdateTime, _ = r.GetTime(6).GoTime(time.UTC) + task.CreateTime, _ = r.GetTime(12).GoTime(time.Local) + task.StartTime, _ = r.GetTime(5).GoTime(time.Local) + task.StateUpdateTime, _ = r.GetTime(6).GoTime(time.Local) return task } @@ -184,22 +194,22 @@ func (stm *TaskManager) executeSQLWithNewSession(ctx context.Context, sql string return } -// AddNewGlobalTask adds a new task to global task table. -func (stm *TaskManager) AddNewGlobalTask(ctx context.Context, key string, tp proto.TaskType, concurrency int, meta []byte) (taskID int64, err error) { +// CreateTask adds a new task to task table. +func (stm *TaskManager) CreateTask(ctx context.Context, key string, tp proto.TaskType, concurrency int, meta []byte) (taskID int64, err error) { err = stm.WithNewSession(func(se sessionctx.Context) error { var err2 error - taskID, err2 = stm.AddGlobalTaskWithSession(ctx, se, key, tp, concurrency, meta) + taskID, err2 = stm.CreateTaskWithSession(ctx, se, key, tp, concurrency, meta) return err2 }) return } -// AddGlobalTaskWithSession adds a new task to global task table with session. -func (*TaskManager) AddGlobalTaskWithSession(ctx context.Context, se sessionctx.Context, key string, tp proto.TaskType, concurrency int, meta []byte) (taskID int64, err error) { - _, err = ExecSQL(ctx, se, - `insert into mysql.tidb_global_task(task_key, type, state, concurrency, step, meta, start_time, state_update_time) - values (%?, %?, %?, %?, %?, %?, CURRENT_TIMESTAMP(), CURRENT_TIMESTAMP())`, - key, tp, proto.TaskStatePending, concurrency, proto.StepInit, meta) +// CreateTaskWithSession adds a new task to task table with session. +func (*TaskManager) CreateTaskWithSession(ctx context.Context, se sessionctx.Context, key string, tp proto.TaskType, concurrency int, meta []byte) (taskID int64, err error) { + _, err = ExecSQL(ctx, se, `insert into mysql.tidb_global_task( + task_key, type, state, priority, concurrency, step, meta, create_time, start_time, state_update_time) + values (%?, %?, %?, %?, %?, %?, %?, CURRENT_TIMESTAMP(), CURRENT_TIMESTAMP(), CURRENT_TIMESTAMP())`, + key, tp, proto.TaskStatePending, proto.NormalPriority, concurrency, proto.StepInit, meta) if err != nil { return 0, err } @@ -215,9 +225,9 @@ func (*TaskManager) AddGlobalTaskWithSession(ctx context.Context, se sessionctx. return taskID, nil } -// GetNewGlobalTask get a new task from global task table, it's used by dispatcher only. -func (stm *TaskManager) GetNewGlobalTask(ctx context.Context) (task *proto.Task, err error) { - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task where state = %? limit 1", proto.TaskStatePending) +// GetOneTask get a task from task table, it's used by dispatcher 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 { return task, err } @@ -226,46 +236,46 @@ func (stm *TaskManager) GetNewGlobalTask(ctx context.Context) (task *proto.Task, return nil, nil } - return row2GlobeTask(rs[0]), nil + return row2Task(rs[0]), nil } -// GetGlobalTasksInStates gets the tasks in the states. -func (stm *TaskManager) GetGlobalTasksInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) { +// GetTasksInStates gets the tasks in the states. +func (stm *TaskManager) GetTasksInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) { if len(states) == 0 { return task, nil } - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task where state in ("+strings.Repeat("%?,", len(states)-1)+"%?)", states...) + rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task where state in ("+strings.Repeat("%?,", len(states)-1)+"%?)", states...) if err != nil { return task, err } for _, r := range rs { - task = append(task, row2GlobeTask(r)) + task = append(task, row2Task(r)) } return task, nil } -// GetGlobalTasksFromHistoryInStates gets the tasks in history table in the states. -func (stm *TaskManager) GetGlobalTasksFromHistoryInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) { +// GetTasksFromHistoryInStates gets the tasks in history table in the states. +func (stm *TaskManager) GetTasksFromHistoryInStates(ctx context.Context, states ...interface{}) (task []*proto.Task, err error) { if len(states) == 0 { return task, nil } - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task_history where state in ("+strings.Repeat("%?,", len(states)-1)+"%?)", states...) + rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task_history where state in ("+strings.Repeat("%?,", len(states)-1)+"%?)", states...) if err != nil { return task, err } for _, r := range rs { - task = append(task, row2GlobeTask(r)) + task = append(task, row2Task(r)) } return task, nil } -// GetGlobalTaskByID gets the task by the global task ID. -func (stm *TaskManager) GetGlobalTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error) { - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task where id = %?", taskID) +// GetTaskByID gets the task by the task ID. +func (stm *TaskManager) GetTaskByID(ctx context.Context, taskID int64) (task *proto.Task, err error) { + rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task where id = %?", taskID) if err != nil { return task, err } @@ -273,13 +283,13 @@ func (stm *TaskManager) GetGlobalTaskByID(ctx context.Context, taskID int64) (ta return nil, nil } - return row2GlobeTask(rs[0]), nil + return row2Task(rs[0]), nil } -// GetTaskByIDWithHistory gets the task by the global task ID from both tidb_global_task and tidb_global_task_history. +// GetTaskByIDWithHistory gets the task by the task ID from both tidb_global_task and tidb_global_task_history. func (stm *TaskManager) GetTaskByIDWithHistory(ctx context.Context, taskID int64) (task *proto.Task, err error) { - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task where id = %? "+ - "union select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task_history where id = %?", taskID, taskID) + rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task where id = %? "+ + "union select "+taskColumns+" from mysql.tidb_global_task_history where id = %?", taskID, taskID) if err != nil { return task, err } @@ -287,12 +297,12 @@ func (stm *TaskManager) GetTaskByIDWithHistory(ctx context.Context, taskID int64 return nil, nil } - return row2GlobeTask(rs[0]), nil + return row2Task(rs[0]), nil } -// GetGlobalTaskByKey gets the task by the task key. -func (stm *TaskManager) GetGlobalTaskByKey(ctx context.Context, key string) (task *proto.Task, err error) { - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task where task_key = %?", key) +// GetTaskByKey gets the task by the task key. +func (stm *TaskManager) GetTaskByKey(ctx context.Context, key string) (task *proto.Task, err error) { + rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task where task_key = %?", key) if err != nil { return task, err } @@ -300,13 +310,13 @@ func (stm *TaskManager) GetGlobalTaskByKey(ctx context.Context, key string) (tas return nil, nil } - return row2GlobeTask(rs[0]), nil + return row2Task(rs[0]), nil } -// GetGlobalTaskByKeyWithHistory gets the task from history table by the task key. -func (stm *TaskManager) GetGlobalTaskByKeyWithHistory(ctx context.Context, key string) (task *proto.Task, err error) { - rs, err := stm.executeSQLWithNewSession(ctx, "select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task where task_key = %?"+ - "union select id, task_key, type, dispatcher_id, state, start_time, state_update_time, meta, concurrency, step, error from mysql.tidb_global_task_history where task_key = %?", key, key) +// GetTaskByKeyWithHistory gets the task from history table by the task key. +func (stm *TaskManager) GetTaskByKeyWithHistory(ctx context.Context, key string) (task *proto.Task, err error) { + rs, err := stm.executeSQLWithNewSession(ctx, "select "+taskColumns+" from mysql.tidb_global_task where task_key = %?"+ + "union select "+taskColumns+" from mysql.tidb_global_task_history where task_key = %?", key, key) if err != nil { return task, err } @@ -314,52 +324,55 @@ func (stm *TaskManager) GetGlobalTaskByKeyWithHistory(ctx context.Context, key s return nil, nil } - return row2GlobeTask(rs[0]), nil + return row2Task(rs[0]), nil } // row2SubTask converts a row to a subtask. func row2SubTask(r chunk.Row) *proto.Subtask { // subtask defines start/update time as bigint, to ensure backward compatible, // we keep it that way, and we convert it here. + createTime, _ := r.GetTime(7).GoTime(time.Local) var startTime, updateTime time.Time - if !r.IsNull(10) { - ts := r.GetInt64(10) + if !r.IsNull(8) { + ts := r.GetInt64(8) startTime = time.Unix(ts, 0) } - if !r.IsNull(11) { - ts := r.GetInt64(11) + if !r.IsNull(9) { + ts := r.GetInt64(9) updateTime = time.Unix(ts, 0) } - task := &proto.Subtask{ + subtask := &proto.Subtask{ ID: r.GetInt64(0), Step: proto.Step(r.GetInt64(1)), - Type: proto.Int2Type(int(r.GetInt64(5))), - SchedulerID: r.GetString(6), - State: proto.TaskState(r.GetString(8)), - Meta: r.GetBytes(12), - Summary: r.GetString(14), + Type: proto.Int2Type(int(r.GetInt64(3))), + ExecID: r.GetString(4), + State: proto.TaskState(r.GetString(5)), + Concurrency: int(r.GetInt64(6)), + CreateTime: createTime, StartTime: startTime, UpdateTime: updateTime, + Meta: r.GetBytes(10), + Summary: r.GetJSON(11).String(), } - tid, err := strconv.Atoi(r.GetString(3)) + taskIDStr := r.GetString(2) + tid, err := strconv.Atoi(taskIDStr) if err != nil { - logutil.BgLogger().Warn("unexpected task ID", zap.String("task ID", r.GetString(3))) + logutil.BgLogger().Warn("unexpected subtask id", zap.String("subtask-id", taskIDStr)) } - task.TaskID = int64(tid) - return task + subtask.TaskID = int64(tid) + return subtask } -// AddNewSubTask adds a new task to subtask table. -func (stm *TaskManager) AddNewSubTask(ctx context.Context, globalTaskID int64, step proto.Step, designatedTiDBID string, meta []byte, tp proto.TaskType, isRevert bool) error { - st := proto.TaskStatePending +// CreateSubTask adds a new task to subtask table. +// used for testing. +func (stm *TaskManager) CreateSubTask(ctx context.Context, taskID int64, step proto.Step, execID string, meta []byte, tp proto.TaskType, isRevert bool) error { + state := proto.TaskStatePending if isRevert { - st = proto.TaskStateRevertPending + state = proto.TaskStateRevertPending } - _, err := stm.executeSQLWithNewSession(ctx, `insert into mysql.tidb_background_subtask - (task_key, step, exec_id, meta, state, type, checkpoint, summary) - values (%?, %?, %?, %?, %?, %?, %?, %?)`, - globalTaskID, step, designatedTiDBID, meta, st, proto.Type2Int(tp), []byte{}, "{}") + _, err := stm.executeSQLWithNewSession(ctx, insertSubtaskBasic+`(%?, %?, %?, %?, %?, %?, 11, CURRENT_TIMESTAMP(), '{}', '{}')`, + step, taskID, execID, meta, state, proto.Type2Int(tp)) if err != nil { return err } @@ -371,7 +384,7 @@ func (stm *TaskManager) AddNewSubTask(ctx context.Context, globalTaskID int64, s func (stm *TaskManager) GetSubtasksInStates(ctx context.Context, tidbID string, taskID int64, step proto.Step, states ...interface{}) ([]*proto.Subtask, error) { args := []interface{}{tidbID, taskID, step} args = append(args, states...) - rs, err := stm.executeSQLWithNewSession(ctx, `select * from mysql.tidb_background_subtask + rs, err := stm.executeSQLWithNewSession(ctx, `select `+subtaskColumns+` from mysql.tidb_background_subtask where exec_id = %? and task_key = %? and step = %? and state in (`+strings.Repeat("%?,", len(states)-1)+"%?)", args...) if err != nil { @@ -391,7 +404,7 @@ func (stm *TaskManager) GetSubtasksByExecIdsAndStepAndState(ctx context.Context, for _, tidbID := range tidbIDs { args = append(args, tidbID) } - rs, err := stm.executeSQLWithNewSession(ctx, `select * from mysql.tidb_background_subtask + rs, err := stm.executeSQLWithNewSession(ctx, `select `+subtaskColumns+` from mysql.tidb_background_subtask where task_key = %? and step = %? and state = %? and exec_id in (`+strings.Repeat("%?,", len(tidbIDs)-1)+"%?)", args...) if err != nil { @@ -409,7 +422,7 @@ func (stm *TaskManager) GetSubtasksByExecIdsAndStepAndState(ctx context.Context, func (stm *TaskManager) GetFirstSubtaskInStates(ctx context.Context, tidbID string, taskID int64, step proto.Step, states ...interface{}) (*proto.Subtask, error) { args := []interface{}{tidbID, taskID, step} args = append(args, states...) - rs, err := stm.executeSQLWithNewSession(ctx, `select * from mysql.tidb_background_subtask + rs, err := stm.executeSQLWithNewSession(ctx, `select `+subtaskColumns+` from mysql.tidb_background_subtask where exec_id = %? and task_key = %? and step = %? and state in (`+strings.Repeat("%?,", len(states)-1)+"%?) limit 1", args...) if err != nil { @@ -445,9 +458,9 @@ func (stm *TaskManager) UpdateErrorToSubtask(ctx context.Context, tidbID string, // PrintSubtaskInfo log the subtask info by taskKey. Only used for UT. func (stm *TaskManager) PrintSubtaskInfo(ctx context.Context, taskID int64) { rs, _ := stm.executeSQLWithNewSession(ctx, - "select * from mysql.tidb_background_subtask_history where task_key = %?", taskID) + `select `+subtaskColumns+` from mysql.tidb_background_subtask_history where task_key = %?`, taskID) rs2, _ := stm.executeSQLWithNewSession(ctx, - "select * from mysql.tidb_background_subtask where task_key = %?", taskID) + `select `+subtaskColumns+` from mysql.tidb_background_subtask where task_key = %?`, taskID) rs = append(rs, rs2...) for _, r := range rs { @@ -468,7 +481,7 @@ func (stm *TaskManager) PrintSubtaskInfo(ctx context.Context, taskID int64) { // GetSubtasksByStepAndState gets the subtask by step and state. func (stm *TaskManager) GetSubtasksByStepAndState(ctx context.Context, taskID int64, step proto.Step, state proto.TaskState) ([]*proto.Subtask, error) { - rs, err := stm.executeSQLWithNewSession(ctx, `select * from mysql.tidb_background_subtask + rs, err := stm.executeSQLWithNewSession(ctx, `select `+subtaskColumns+` from mysql.tidb_background_subtask where task_key = %? and state = %? and step = %?`, taskID, state, step) if err != nil { @@ -595,7 +608,7 @@ func (stm *TaskManager) FinishSubtask(ctx context.Context, tidbID string, id int return err } -// DeleteSubtasksByTaskID deletes the subtask of the given global task ID. +// DeleteSubtasksByTaskID deletes the subtask of the given task ID. func (stm *TaskManager) DeleteSubtasksByTaskID(ctx context.Context, taskID int64) error { _, err := stm.executeSQLWithNewSession(ctx, `delete from mysql.tidb_background_subtask where task_key = %?`, taskID) @@ -606,7 +619,7 @@ func (stm *TaskManager) DeleteSubtasksByTaskID(ctx context.Context, taskID int64 return nil } -// GetSchedulerIDsByTaskID gets the scheduler IDs of the given global task ID. +// GetSchedulerIDsByTaskID gets the scheduler IDs of the given task ID. func (stm *TaskManager) GetSchedulerIDsByTaskID(ctx context.Context, taskID int64) ([]string, error) { rs, err := stm.executeSQLWithNewSession(ctx, `select distinct(exec_id) from mysql.tidb_background_subtask where task_key = %?`, taskID) @@ -626,7 +639,7 @@ func (stm *TaskManager) GetSchedulerIDsByTaskID(ctx context.Context, taskID int6 return instanceIDs, nil } -// GetSchedulerIDsByTaskIDAndStep gets the scheduler IDs of the given global task ID and step. +// GetSchedulerIDsByTaskIDAndStep gets the scheduler IDs of the given task ID and step. func (stm *TaskManager) GetSchedulerIDsByTaskIDAndStep(ctx context.Context, taskID int64, step proto.Step) ([]string, error) { rs, err := stm.executeSQLWithNewSession(ctx, `select distinct(exec_id) from mysql.tidb_background_subtask where task_key = %? and step = %?`, taskID, step) @@ -665,7 +678,7 @@ func (stm *TaskManager) UpdateSubtasksSchedulerIDs(ctx context.Context, taskID i for _, subtask := range subtasks { _, err := ExecSQL(ctx, se, "update mysql.tidb_background_subtask set exec_id = %? where id = %? and state = %? and task_key = %?", - subtask.SchedulerID, + subtask.ExecID, subtask.ID, subtask.State, taskID) @@ -714,14 +727,14 @@ func (stm *TaskManager) ResumeSubtasks(ctx context.Context, taskID int64) error return err } -// UpdateGlobalTaskAndAddSubTasks update the global task and add new subtasks -func (stm *TaskManager) UpdateGlobalTaskAndAddSubTasks(ctx context.Context, gTask *proto.Task, subtasks []*proto.Subtask, prevState proto.TaskState) (bool, error) { +// UpdateTaskAndAddSubTasks update the task and add new subtasks +func (stm *TaskManager) UpdateTaskAndAddSubTasks(ctx context.Context, task *proto.Task, subtasks []*proto.Subtask, prevState proto.TaskState) (bool, error) { retryable := true err := stm.WithNewTxn(ctx, func(se sessionctx.Context) error { _, 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 = %?", - gTask.State, gTask.DispatcherID, gTask.Step, gTask.Concurrency, gTask.Meta, serializeErr(gTask.Error), gTask.ID, prevState) + task.State, task.DispatcherID, task.Step, task.Concurrency, task.Meta, serializeErr(task.Error), task.ID, prevState) if err != nil { return err } @@ -734,14 +747,14 @@ func (stm *TaskManager) UpdateGlobalTaskAndAddSubTasks(ctx context.Context, gTas } // 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 global task and insert subtasks in one Txn. + // So when dispatching 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. // // For now, we update nothing in proto.Task in UT's OnNextSubtasksBatch, so the AffectedRows will be 0. So UT can't fully compatible - // with current UpdateGlobalTaskAndAddSubTasks implementation. - rs, err := ExecSQL(ctx, se, "select id from mysql.tidb_global_task where id = %? and state = %?", gTask.ID, prevState) + // with current UpdateTaskAndAddSubTasks implementation. + rs, err := ExecSQL(ctx, se, "select id from mysql.tidb_global_task where id = %? and state = %?", task.ID, prevState) if err != nil { return err } @@ -759,13 +772,12 @@ func (stm *TaskManager) UpdateGlobalTaskAndAddSubTasks(ctx context.Context, gTas }) if len(subtasks) > 0 { subtaskState := proto.TaskStatePending - if gTask.State == proto.TaskStateReverting { + if task.State == proto.TaskStateReverting { subtaskState = proto.TaskStateRevertPending } sql := new(strings.Builder) - if err := sqlescape.FormatSQL(sql, "insert into mysql.tidb_background_subtask \n"+ - "(step, task_key, exec_id, meta, state, type, checkpoint, summary) values "); err != nil { + if err := sqlescape.FormatSQL(sql, insertSubtaskBasic); err != nil { return err } for i, subtask := range subtasks { @@ -774,8 +786,8 @@ func (stm *TaskManager) UpdateGlobalTaskAndAddSubTasks(ctx context.Context, gTas return err } } - if err := sqlescape.FormatSQL(sql, "(%?, %?, %?, %?, %?, %?, %?, %?)", - subtask.Step, gTask.ID, subtask.SchedulerID, subtask.Meta, subtaskState, proto.Type2Int(subtask.Type), []byte{}, "{}"); err != nil { + if err := sqlescape.FormatSQL(sql, "(%?, %?, %?, %?, %?, %?, %?, CURRENT_TIMESTAMP(), '{}', '{}')", + subtask.Step, task.ID, subtask.ExecID, subtask.Meta, subtaskState, proto.Type2Int(subtask.Type), subtask.Concurrency); err != nil { return err } } @@ -806,8 +818,8 @@ func serializeErr(err error) []byte { return errBytes } -// CancelGlobalTask cancels global task. -func (stm *TaskManager) CancelGlobalTask(ctx context.Context, taskID int64) error { +// CancelTask cancels task. +func (stm *TaskManager) CancelTask(ctx context.Context, taskID int64) error { _, err := stm.executeSQLWithNewSession(ctx, "update mysql.tidb_global_task set state=%?, state_update_time = CURRENT_TIMESTAMP() "+ "where id=%? and state in (%?, %?)", @@ -816,8 +828,8 @@ func (stm *TaskManager) CancelGlobalTask(ctx context.Context, taskID int64) erro return err } -// CancelGlobalTaskByKeySession cancels global task by key using input session. -func (*TaskManager) CancelGlobalTaskByKeySession(ctx context.Context, se sessionctx.Context, taskKey string) error { +// CancelTaskByKeySession cancels task by key using input session. +func (*TaskManager) CancelTaskByKeySession(ctx context.Context, se sessionctx.Context, taskKey string) error { _, err := ExecSQL(ctx, se, "update mysql.tidb_global_task set state=%?, state_update_time = CURRENT_TIMESTAMP() "+ "where task_key=%? and state in (%?, %?)", @@ -825,8 +837,8 @@ func (*TaskManager) CancelGlobalTaskByKeySession(ctx context.Context, se session return err } -// IsGlobalTaskCancelling checks whether the task state is cancelling. -func (stm *TaskManager) IsGlobalTaskCancelling(ctx context.Context, taskID int64) (bool, error) { +// IsTaskCancelling checks whether the task state is cancelling. +func (stm *TaskManager) IsTaskCancelling(ctx context.Context, taskID int64) (bool, error) { rs, err := stm.executeSQLWithNewSession(ctx, "select 1 from mysql.tidb_global_task where id=%? and state = %?", taskID, proto.TaskStateCancelling, ) @@ -892,7 +904,7 @@ func (stm *TaskManager) GetSubtasksForImportInto(ctx context.Context, taskID int ) err = stm.WithNewTxn(ctx, func(se sessionctx.Context) error { rs, err = ExecSQL(ctx, se, - "select * from mysql.tidb_background_subtask where task_key = %? and step = %?", + `select `+subtaskColumns+` from mysql.tidb_background_subtask where task_key = %? and step = %?`, taskID, step, ) if err != nil { @@ -902,7 +914,7 @@ func (stm *TaskManager) GetSubtasksForImportInto(ctx context.Context, taskID int // To avoid the situation that the subtasks has been `TransferSubTasks2History` // when the user show import jobs, we need to check the history table. rsFromHistory, err := ExecSQL(ctx, se, - "select * from mysql.tidb_background_subtask_history where task_key = %? and step = %?", + `select `+subtaskColumns+` from mysql.tidb_background_subtask_history where task_key = %? and step = %?`, taskID, step, ) if err != nil { @@ -929,7 +941,7 @@ func (stm *TaskManager) GetSubtasksForImportInto(ctx context.Context, taskID int // TransferSubTasks2History move all the finished subTask to tidb_background_subtask_history by taskID func (stm *TaskManager) TransferSubTasks2History(ctx context.Context, taskID int64) error { return stm.WithNewTxn(ctx, func(se sessionctx.Context) error { - _, err := ExecSQL(ctx, se, "insert into mysql.tidb_background_subtask_history select * from mysql.tidb_background_subtask where task_key = %?", taskID) + _, err := ExecSQL(ctx, se, `insert into mysql.tidb_background_subtask_history select * from mysql.tidb_background_subtask where task_key = %?`, taskID) if err != nil { return err } @@ -963,7 +975,7 @@ func (stm *TaskManager) TransferTasks2History(ctx context.Context, tasks []*prot return stm.WithNewTxn(ctx, func(se sessionctx.Context) error { insertSQL := new(strings.Builder) if err := sqlescape.FormatSQL(insertSQL, "replace into mysql.tidb_global_task_history"+ - "(id, task_key, type, dispatcher_id, state, start_time, state_update_time,"+ + "(id, task_key, type, dispatcher_id, state, priority, start_time, state_update_time,"+ "meta, concurrency, step, error) values"); err != nil { return err } @@ -974,9 +986,9 @@ func (stm *TaskManager) TransferTasks2History(ctx context.Context, tasks []*prot return err } } - if err := sqlescape.FormatSQL(insertSQL, "(%?, %?, %?, %?, %?, %?, %?, %?, %?, %?, %?)", + if err := sqlescape.FormatSQL(insertSQL, "(%?, %?, %?, %?, %?, %?, %?, %?, %?, %?, %?, %?)", task.ID, task.Key, task.Type, task.DispatcherID, - task.State, task.StartTime, task.StateUpdateTime, + task.State, task.Priority, task.StartTime, task.StateUpdateTime, task.Meta, task.Concurrency, task.Step, serializeErr(task.Error)); err != nil { return err } diff --git a/pkg/disttask/framework/storage/util.go b/pkg/disttask/framework/storage/util.go index 9784dfb121c28..3a5bc97f9675b 100644 --- a/pkg/disttask/framework/storage/util.go +++ b/pkg/disttask/framework/storage/util.go @@ -33,7 +33,7 @@ func GetSubtasksFromHistoryForTest(ctx context.Context, stm *TaskManager) (int, // GetSubtasksFromHistoryByTaskIDForTest gets subtasks by taskID from history table for test. func GetSubtasksFromHistoryByTaskIDForTest(ctx context.Context, stm *TaskManager, taskID int64) (int, error) { rs, err := stm.executeSQLWithNewSession(ctx, - "select * from mysql.tidb_background_subtask_history where task_key = %?", taskID) + `select `+subtaskColumns+` from mysql.tidb_background_subtask_history where task_key = %?`, taskID) if err != nil { return 0, err } @@ -43,7 +43,7 @@ func GetSubtasksFromHistoryByTaskIDForTest(ctx context.Context, stm *TaskManager // GetSubtasksByTaskIDForTest gets subtasks by taskID for test. func GetSubtasksByTaskIDForTest(ctx context.Context, stm *TaskManager, taskID int64) ([]*proto.Subtask, error) { rs, err := stm.executeSQLWithNewSession(ctx, - "select * from mysql.tidb_background_subtask where task_key = %?", taskID) + `select `+subtaskColumns+` from mysql.tidb_background_subtask where task_key = %?`, taskID) if err != nil { return nil, err } diff --git a/pkg/disttask/framework/testutil/dispatcher_util.go b/pkg/disttask/framework/testutil/dispatcher_util.go index 748ca9404f3c2..8399f188d869f 100644 --- a/pkg/disttask/framework/testutil/dispatcher_util.go +++ b/pkg/disttask/framework/testutil/dispatcher_util.go @@ -48,15 +48,15 @@ func GetMockBasicDispatcherExt(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, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if gTask.Step == proto.StepInit { + 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 gTask.Step == proto.StepOne { + if task.Step == proto.StepOne { return [][]byte{ []byte("task4"), }, nil @@ -92,8 +92,8 @@ func GetMockHATestDispatcherExt(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, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if gTask.Step == proto.StepInit { + 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"), @@ -107,7 +107,7 @@ func GetMockHATestDispatcherExt(ctrl *gomock.Controller) dispatcher.Extension { []byte("task10"), }, nil } - if gTask.Step == proto.StepOne { + if task.Step == proto.StepOne { return [][]byte{ []byte("task11"), []byte("task12"), @@ -154,7 +154,7 @@ func GetPlanNotRetryableErrDispatcherExt(ctrl *gomock.Controller) dispatcher.Ext }, ).AnyTimes() mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { + func(_ context.Context, _ dispatcher.TaskHandle, _ *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { return nil, errors.New("not retryable err") }, ).AnyTimes() @@ -186,8 +186,8 @@ func GetPlanErrDispatcherExt(ctrl *gomock.Controller, testContext *TestContext) }, ).AnyTimes() mockDispatcher.EXPECT().OnNextSubtasksBatch(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).DoAndReturn( - func(_ context.Context, _ dispatcher.TaskHandle, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if gTask.Step == proto.StepInit { + 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") @@ -198,7 +198,7 @@ func GetPlanErrDispatcherExt(ctrl *gomock.Controller, testContext *TestContext) []byte("task3"), }, nil } - if gTask.Step == proto.StepOne { + if task.Step == proto.StepOne { return [][]byte{ []byte("task4"), }, nil @@ -235,8 +235,8 @@ func GetMockRollbackDispatcherExt(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, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if gTask.Step == proto.StepInit { + 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"), @@ -274,8 +274,8 @@ func GetMockDynamicDispatchExt(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, gTask *proto.Task, _ []*infosync.ServerInfo, _ proto.Step) (metas [][]byte, err error) { - if gTask.Step == proto.StepInit { + 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"), @@ -283,7 +283,7 @@ func GetMockDynamicDispatchExt(ctrl *gomock.Controller) dispatcher.Extension { } // step2 - if gTask.Step == proto.StepOne { + if task.Step == proto.StepOne { return [][]byte{ []byte("task"), }, nil diff --git a/pkg/disttask/framework/testutil/disttest_util.go b/pkg/disttask/framework/testutil/disttest_util.go index d60a34cfd28a6..09353a82b5d1c 100644 --- a/pkg/disttask/framework/testutil/disttest_util.go +++ b/pkg/disttask/framework/testutil/disttest_util.go @@ -117,7 +117,7 @@ func RegisterRollbackTaskMeta(t *testing.T, ctrl *gomock.Controller, mockDispatc func DispatchTask(ctx context.Context, t *testing.T, taskKey string) *proto.Task { mgr, err := storage.GetTaskManager() require.NoError(t, err) - _, err = mgr.AddNewGlobalTask(ctx, taskKey, proto.TaskTypeExample, 8, nil) + _, err = mgr.CreateTask(ctx, taskKey, proto.TaskTypeExample, 8, nil) require.NoError(t, err) return WaitTaskExit(ctx, t, taskKey) } @@ -134,7 +134,7 @@ func WaitTaskExit(ctx context.Context, t *testing.T, taskKey string) *proto.Task } time.Sleep(time.Second) - task, err = mgr.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err = mgr.GetTaskByKeyWithHistory(ctx, taskKey) require.NoError(t, err) require.NotNil(t, task) if task.State != proto.TaskStatePending && task.State != proto.TaskStateRunning && task.State != proto.TaskStateCancelling && task.State != proto.TaskStateReverting && task.State != proto.TaskStatePausing { @@ -194,7 +194,7 @@ func DispatchMultiTasksAndOneFail(ctx context.Context, t *testing.T, num int, te require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/MockExecutorRunErr", "1*return(true)")) for i := 0; i < num; i++ { - _, err = mgr.AddNewGlobalTask(ctx, fmt.Sprintf("key%d", i), proto.TaskTypeExample, 8, nil) + _, err = mgr.CreateTask(ctx, fmt.Sprintf("key%d", i), proto.TaskTypeExample, 8, nil) require.NoError(t, err) } for i := 0; i < num; i++ { diff --git a/pkg/disttask/importinto/dispatcher.go b/pkg/disttask/importinto/dispatcher.go index fe6b7e23c5a96..556f932f01b4e 100644 --- a/pkg/disttask/importinto/dispatcher.go +++ b/pkg/disttask/importinto/dispatcher.go @@ -200,19 +200,19 @@ func (dsp *ImportDispatcherExt) unregisterTask(ctx context.Context, task *proto. func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( ctx context.Context, taskHandle dispatcher.TaskHandle, - gTask *proto.Task, + task *proto.Task, serverInfos []*infosync.ServerInfo, nextStep proto.Step, ) ( resSubtaskMeta [][]byte, err error) { logger := logutil.BgLogger().With( - zap.Stringer("type", gTask.Type), - zap.Int64("task-id", gTask.ID), - zap.String("curr-step", stepStr(gTask.Step)), + zap.Stringer("type", task.Type), + zap.Int64("task-id", task.ID), + zap.String("curr-step", stepStr(task.Step)), zap.String("next-step", stepStr(nextStep)), ) taskMeta := &TaskMeta{} - err = json.Unmarshal(gTask.Meta, taskMeta) + err = json.Unmarshal(task.Meta, taskMeta) if err != nil { return nil, errors.Trace(err) } @@ -232,7 +232,7 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( return nil, err } case StepMergeSort: - sortAndEncodeMeta, err := taskHandle.GetPreviousSubtaskMetas(gTask.ID, StepEncodeAndSort) + sortAndEncodeMeta, err := taskHandle.GetPreviousSubtaskMetas(task.ID, StepEncodeAndSort) if err != nil { return nil, err } @@ -243,11 +243,11 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( }) // merge sort might be skipped for some kv groups, so we need to get all // subtask metas of StepEncodeAndSort step too. - encodeAndSortMetas, err := taskHandle.GetPreviousSubtaskMetas(gTask.ID, StepEncodeAndSort) + encodeAndSortMetas, err := taskHandle.GetPreviousSubtaskMetas(task.ID, StepEncodeAndSort) if err != nil { return nil, err } - mergeSortMetas, err := taskHandle.GetPreviousSubtaskMetas(gTask.ID, StepMergeSort) + mergeSortMetas, err := taskHandle.GetPreviousSubtaskMetas(task.ID, StepMergeSort) if err != nil { return nil, err } @@ -257,7 +257,7 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( return nil, err } case StepPostProcess: - dsp.switchTiKV2NormalMode(ctx, gTask, logger) + dsp.switchTiKV2NormalMode(ctx, task, logger) failpoint.Inject("clearLastSwitchTime", func() { dsp.lastSwitchTime.Store(time.Time{}) }) @@ -268,11 +268,11 @@ 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, gTask, taskMeta, dsp.GlobalSort); err != nil { + if err := updateResult(taskHandle, task, taskMeta, dsp.GlobalSort); err != nil { return nil, err } step := getStepOfEncode(dsp.GlobalSort) - metas, err := taskHandle.GetPreviousSubtaskMetas(gTask.ID, step) + metas, err := taskHandle.GetPreviousSubtaskMetas(task.ID, step) if err != nil { return nil, err } @@ -281,19 +281,19 @@ func (dsp *ImportDispatcherExt) OnNextSubtasksBatch( case proto.StepDone: return nil, nil default: - return nil, errors.Errorf("unknown step %d", gTask.Step) + return nil, errors.Errorf("unknown step %d", task.Step) } planCtx := planner.PlanCtx{ Ctx: ctx, - TaskID: gTask.ID, + TaskID: task.ID, PreviousSubtaskMetas: previousSubtaskMetas, GlobalSort: dsp.GlobalSort, NextTaskStep: nextStep, ExecuteNodesCnt: len(serverInfos), } logicalPlan := &LogicalPlan{} - if err := logicalPlan.FromTaskMeta(gTask.Meta); err != nil { + if err := logicalPlan.FromTaskMeta(task.Meta); err != nil { return nil, err } physicalPlan, err := logicalPlan.ToPhysicalPlan(planCtx) @@ -331,9 +331,9 @@ func (dsp *ImportDispatcherExt) OnDone(ctx context.Context, handle dispatcher.Ta } // GetEligibleInstances implements dispatcher.Extension interface. -func (*ImportDispatcherExt) GetEligibleInstances(ctx context.Context, gTask *proto.Task) ([]*infosync.ServerInfo, bool, error) { +func (*ImportDispatcherExt) GetEligibleInstances(ctx context.Context, task *proto.Task) ([]*infosync.ServerInfo, bool, error) { taskMeta := &TaskMeta{} - err := json.Unmarshal(gTask.Meta, taskMeta) + err := json.Unmarshal(task.Meta, taskMeta) if err != nil { return nil, true, errors.Trace(err) } @@ -506,12 +506,12 @@ func executeSQL(ctx context.Context, executor storage.SessionExecutor, logger *z }) } -func updateMeta(gTask *proto.Task, taskMeta *TaskMeta) error { +func updateMeta(task *proto.Task, taskMeta *TaskMeta) error { bs, err := json.Marshal(taskMeta) if err != nil { return errors.Trace(err) } - gTask.Meta = bs + task.Meta = bs return nil } @@ -535,10 +535,10 @@ func getStepOfEncode(globalSort bool) proto.Step { return StepImport } -// we will update taskMeta in place and make gTask.Meta point to the new taskMeta. -func updateResult(handle dispatcher.TaskHandle, gTask *proto.Task, taskMeta *TaskMeta, globalSort bool) error { +// 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 { stepOfEncode := getStepOfEncode(globalSort) - metas, err := handle.GetPreviousSubtaskMetas(gTask.ID, stepOfEncode) + metas, err := handle.GetPreviousSubtaskMetas(task.ID, stepOfEncode) if err != nil { return err } @@ -561,17 +561,17 @@ func updateResult(handle dispatcher.TaskHandle, gTask *proto.Task, taskMeta *Tas taskMeta.Result.ColSizeMap = columnSizeMap if globalSort { - taskMeta.Result.LoadedRowCnt, err = getLoadedRowCountOnGlobalSort(handle, gTask) + taskMeta.Result.LoadedRowCnt, err = getLoadedRowCountOnGlobalSort(handle, task) if err != nil { return err } } - return updateMeta(gTask, taskMeta) + return updateMeta(task, taskMeta) } -func getLoadedRowCountOnGlobalSort(handle dispatcher.TaskHandle, gTask *proto.Task) (uint64, error) { - metas, err := handle.GetPreviousSubtaskMetas(gTask.ID, StepWriteAndIngest) +func getLoadedRowCountOnGlobalSort(handle dispatcher.TaskHandle, task *proto.Task) (uint64, error) { + metas, err := handle.GetPreviousSubtaskMetas(task.ID, StepWriteAndIngest) if err != nil { return 0, err } @@ -612,17 +612,17 @@ func startJob(ctx context.Context, logger *zap.Logger, taskHandle dispatcher.Tas } func job2Step(ctx context.Context, logger *zap.Logger, taskMeta *TaskMeta, step string) error { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() if err != nil { return err } // todo: use dispatcher.TaskHandle - // we might call this in scheduler later, there's no dispatcher.TaskHandle, so we use globalTaskManager here. + // we might call this in scheduler later, there's no dispatcher.TaskHandle, 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, func(ctx context.Context) (bool, error) { - return true, globalTaskManager.WithNewSession(func(se sessionctx.Context) error { + return true, taskManager.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) return importer.Job2Step(ctx, exec, taskMeta.JobID, step) }) @@ -631,9 +631,9 @@ func job2Step(ctx context.Context, logger *zap.Logger, taskMeta *TaskMeta, step } func (dsp *ImportDispatcherExt) finishJob(ctx context.Context, logger *zap.Logger, - taskHandle dispatcher.TaskHandle, gTask *proto.Task, taskMeta *TaskMeta) error { + taskHandle dispatcher.TaskHandle, task *proto.Task, taskMeta *TaskMeta) error { // we have already switch import-mode when switch to post-process step. - dsp.unregisterTask(ctx, gTask) + dsp.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) @@ -647,10 +647,10 @@ func (dsp *ImportDispatcherExt) finishJob(ctx context.Context, logger *zap.Logge ) } -func (dsp *ImportDispatcherExt) failJob(ctx context.Context, taskHandle dispatcher.TaskHandle, gTask *proto.Task, +func (dsp *ImportDispatcherExt) failJob(ctx context.Context, taskHandle dispatcher.TaskHandle, task *proto.Task, taskMeta *TaskMeta, logger *zap.Logger, errorMsg string) error { - dsp.switchTiKV2NormalMode(ctx, gTask, logger) - dsp.unregisterTask(ctx, gTask) + dsp.switchTiKV2NormalMode(ctx, task, logger) + dsp.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, @@ -679,13 +679,13 @@ func (dsp *ImportDispatcherExt) cancelJob(ctx context.Context, taskHandle dispat ) } -func redactSensitiveInfo(gTask *proto.Task, taskMeta *TaskMeta) { +func redactSensitiveInfo(task *proto.Task, taskMeta *TaskMeta) { taskMeta.Stmt = "" taskMeta.Plan.Path = ast.RedactURL(taskMeta.Plan.Path) if taskMeta.Plan.CloudStorageURI != "" { taskMeta.Plan.CloudStorageURI = ast.RedactURL(taskMeta.Plan.CloudStorageURI) } - if err := updateMeta(gTask, taskMeta); err != nil { + if err := updateMeta(task, taskMeta); err != nil { // marshal failed, should not happen logutil.BgLogger().Warn("failed to update task meta", zap.Error(err)) } diff --git a/pkg/disttask/importinto/dispatcher_test.go b/pkg/disttask/importinto/dispatcher_test.go index 59148930c4a23..839dd14cf8e7d 100644 --- a/pkg/disttask/importinto/dispatcher_test.go +++ b/pkg/disttask/importinto/dispatcher_test.go @@ -62,10 +62,10 @@ func (s *importIntoSuite) TestDispatcherGetEligibleInstances() { mockedAllServerInfos := makeFailpointRes(serverInfoMap) dsp := ImportDispatcherExt{} - gTask := &proto.Task{Meta: []byte("{}")} + 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, gTask) + eligibleInstances, _, err := dsp.GetEligibleInstances(ctx, task) s.NoError(err) // order of slice is not stable, change to map resultMap := map[string]*infosync.ServerInfo{} @@ -74,8 +74,8 @@ func (s *importIntoSuite) TestDispatcherGetEligibleInstances() { } s.Equal(serverInfoMap, resultMap) - gTask.Meta = []byte(`{"EligibleInstances":[{"ip": "1.1.1.1", "listening_port": 4000}]}`) - eligibleInstances, _, err = dsp.GetEligibleInstances(ctx, gTask) + task.Meta = []byte(`{"EligibleInstances":[{"ip": "1.1.1.1", "listening_port": 4000}]}`) + eligibleInstances, _, err = dsp.GetEligibleInstances(ctx, task) s.NoError(err) s.Equal([]*infosync.ServerInfo{{IP: "1.1.1.1", Port: 4000}}, eligibleInstances) } diff --git a/pkg/disttask/importinto/dispatcher_testkit_test.go b/pkg/disttask/importinto/dispatcher_testkit_test.go index e06f5e71968f3..6e2e6ee2225e0 100644 --- a/pkg/disttask/importinto/dispatcher_testkit_test.go +++ b/pkg/disttask/importinto/dispatcher_testkit_test.go @@ -85,7 +85,7 @@ func TestDispatcherExtLocalSort(t *testing.T) { require.NoError(t, err) taskMeta, err := json.Marshal(task) require.NoError(t, err) - taskID, err := manager.AddNewGlobalTask(ctx, importinto.TaskKey(jobID), proto.ImportInto, 1, taskMeta) + taskID, err := manager.CreateTask(ctx, importinto.TaskKey(jobID), proto.ImportInto, 1, taskMeta) require.NoError(t, err) task.ID = taskID @@ -105,14 +105,14 @@ func TestDispatcherExtLocalSort(t *testing.T) { // update task/subtask, and finish subtask, so we can go to next stage subtasks := make([]*proto.Subtask, 0, len(subtaskMetas)) for _, m := range subtaskMetas { - subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", m)) + subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", 1, m)) } - _, err = manager.UpdateGlobalTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) + _, err = manager.UpdateTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) require.NoError(t, err) gotSubtasks, err := manager.GetSubtasksForImportInto(ctx, taskID, importinto.StepImport) require.NoError(t, err) for _, s := range gotSubtasks { - require.NoError(t, manager.FinishSubtask(ctx, s.SchedulerID, s.ID, []byte("{}"))) + require.NoError(t, manager.FinishSubtask(ctx, s.ExecID, s.ID, []byte("{}"))) } // to post-process stage, job should be running and in validating step subtaskMetas, err = ext.OnNextSubtasksBatch(ctx, d, task, serverInfos, ext.GetNextStep(task)) @@ -230,7 +230,7 @@ func TestDispatcherExtGlobalSort(t *testing.T) { require.NoError(t, err) taskMeta, err := json.Marshal(task) require.NoError(t, err) - taskID, err := manager.AddNewGlobalTask(ctx, importinto.TaskKey(jobID), proto.ImportInto, 1, taskMeta) + taskID, err := manager.CreateTask(ctx, importinto.TaskKey(jobID), proto.ImportInto, 1, taskMeta) require.NoError(t, err) task.ID = taskID @@ -253,9 +253,9 @@ func TestDispatcherExtGlobalSort(t *testing.T) { // update task/subtask, and finish subtask, so we can go to next stage subtasks := make([]*proto.Subtask, 0, len(subtaskMetas)) for _, m := range subtaskMetas { - subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", m)) + subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", 1, m)) } - _, err = manager.UpdateGlobalTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) + _, err = manager.UpdateTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) require.NoError(t, err) gotSubtasks, err := manager.GetSubtasksForImportInto(ctx, taskID, task.Step) require.NoError(t, err) @@ -290,7 +290,7 @@ func TestDispatcherExtGlobalSort(t *testing.T) { sortStepMetaBytes, err := json.Marshal(sortStepMeta) require.NoError(t, err) for _, s := range gotSubtasks { - require.NoError(t, manager.FinishSubtask(ctx, s.SchedulerID, s.ID, sortStepMetaBytes)) + require.NoError(t, manager.FinishSubtask(ctx, s.ExecID, s.ID, sortStepMetaBytes)) } // to merge-sort stage @@ -310,9 +310,9 @@ func TestDispatcherExtGlobalSort(t *testing.T) { // update task/subtask, and finish subtask, so we can go to next stage subtasks = make([]*proto.Subtask, 0, len(subtaskMetas)) for _, m := range subtaskMetas { - subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", m)) + subtasks = append(subtasks, proto.NewSubtask(task.Step, task.ID, task.Type, "", 1, m)) } - _, err = manager.UpdateGlobalTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) + _, err = manager.UpdateTaskAndAddSubTasks(ctx, task, subtasks, proto.TaskStatePending) require.NoError(t, err) gotSubtasks, err = manager.GetSubtasksForImportInto(ctx, taskID, task.Step) require.NoError(t, err) @@ -328,7 +328,7 @@ func TestDispatcherExtGlobalSort(t *testing.T) { mergeSortStepMetaBytes, err := json.Marshal(mergeSortStepMeta) require.NoError(t, err) for _, s := range gotSubtasks { - require.NoError(t, manager.FinishSubtask(ctx, s.SchedulerID, s.ID, mergeSortStepMetaBytes)) + require.NoError(t, manager.FinishSubtask(ctx, s.ExecID, s.ID, mergeSortStepMetaBytes)) } // to write-and-ingest stage diff --git a/pkg/disttask/importinto/job.go b/pkg/disttask/importinto/job.go index 54053397cceb1..dd90e9bae2361 100644 --- a/pkg/disttask/importinto/job.go +++ b/pkg/disttask/importinto/job.go @@ -109,7 +109,7 @@ func (ti *DistImporter) ImportTask(task *proto.Task) { ti.Group.Go(func() error { defer close(ti.Done) // task is run using distribute framework, so we only wait for the task to finish. - return handle.WaitGlobalTask(ti.GroupCtx, task.ID) + return handle.WaitTask(ti.GroupCtx, task.ID) }) } @@ -138,8 +138,8 @@ func (ti *DistImporter) SubmitTask(ctx context.Context) (int64, *proto.Task, err if ti.instance != nil { instances = append(instances, ti.instance) } - // we use globalTaskManager to submit task, user might not have the privilege to system tables. - globalTaskManager, err := storage.GetTaskManager() + // we use taskManager to submit task, user might not have the privilege to system tables. + taskManager, err := storage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return 0, nil, err @@ -147,7 +147,7 @@ func (ti *DistImporter) SubmitTask(ctx context.Context) (int64, *proto.Task, err var jobID, taskID int64 plan := ti.plan - if err = globalTaskManager.WithNewTxn(ctx, func(se sessionctx.Context) error { + if err = taskManager.WithNewTxn(ctx, func(se sessionctx.Context) error { var err2 error exec := se.(sqlexec.SQLExecutor) // If 2 client try to execute IMPORT INTO concurrently, there's chance that both of them will pass the check. @@ -193,24 +193,24 @@ func (ti *DistImporter) SubmitTask(ctx context.Context) (int64, *proto.Task, err }); err != nil { return 0, nil, err } - globalTask, err := globalTaskManager.GetGlobalTaskByID(ctx, taskID) + task, err := taskManager.GetTaskByID(ctx, taskID) if err != nil { return 0, nil, err } - if globalTask == nil { - return 0, nil, errors.Errorf("cannot find global task with ID %d", taskID) + if task == nil { + return 0, nil, errors.Errorf("cannot find task with ID %d", taskID) } - metrics.UpdateMetricsForAddTask(globalTask) + metrics.UpdateMetricsForAddTask(task) // update logger with task id. ti.jobID = jobID ti.taskID = taskID - ti.logger = ti.logger.With(zap.Int64("task-id", globalTask.ID)) + ti.logger = ti.logger.With(zap.Int64("task-id", task.ID)) - ti.logger.Info("job submitted to global task queue", + ti.logger.Info("job submitted to task queue", zap.Int64("job-id", jobID), zap.Int64("thread-cnt", plan.ThreadCnt)) - return jobID, globalTask, nil + return jobID, task, nil } func (*DistImporter) taskKey() string { @@ -224,21 +224,21 @@ func (ti *DistImporter) JobID() int64 { } func getTaskMeta(ctx context.Context, jobID int64) (*TaskMeta, error) { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return nil, err } taskKey := TaskKey(jobID) - globalTask, err := globalTaskManager.GetGlobalTaskByKey(ctx, taskKey) + task, err := taskManager.GetTaskByKey(ctx, taskKey) if err != nil { return nil, err } - if globalTask == nil { - return nil, errors.Errorf("cannot find global task with key %s", taskKey) + if task == nil { + return nil, errors.Errorf("cannot find task with key %s", taskKey) } var taskMeta TaskMeta - if err := json.Unmarshal(globalTask.Meta, &taskMeta); err != nil { + if err := json.Unmarshal(task.Meta, &taskMeta); err != nil { return nil, errors.Trace(err) } return &taskMeta, nil @@ -247,18 +247,18 @@ func getTaskMeta(ctx context.Context, jobID int64) (*TaskMeta, error) { // GetTaskImportedRows gets the number of imported rows of a job. // Note: for finished job, we can get the number of imported rows from task meta. func GetTaskImportedRows(ctx context.Context, jobID int64) (uint64, error) { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return 0, err } taskKey := TaskKey(jobID) - task, err := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) if err != nil { return 0, err } if task == nil { - return 0, errors.Errorf("cannot find global task with key %s", taskKey) + return 0, errors.Errorf("cannot find task with key %s", taskKey) } taskMeta := TaskMeta{} if err = json.Unmarshal(task.Meta, &taskMeta); err != nil { @@ -266,7 +266,7 @@ func GetTaskImportedRows(ctx context.Context, jobID int64) (uint64, error) { } var importedRows uint64 if taskMeta.Plan.CloudStorageURI == "" { - subtasks, err := globalTaskManager.GetSubtasksForImportInto(ctx, task.ID, StepImport) + subtasks, err := taskManager.GetSubtasksForImportInto(ctx, task.ID, StepImport) if err != nil { return 0, err } @@ -278,7 +278,7 @@ func GetTaskImportedRows(ctx context.Context, jobID int64) (uint64, error) { importedRows += subtaskMeta.Result.LoadedRowCnt } } else { - subtasks, err := globalTaskManager.GetSubtasksForImportInto(ctx, task.ID, StepWriteAndIngest) + subtasks, err := taskManager.GetSubtasksForImportInto(ctx, task.ID, StepWriteAndIngest) if err != nil { return 0, err } diff --git a/pkg/disttask/importinto/job_testkit_test.go b/pkg/disttask/importinto/job_testkit_test.go index f9583b72e5989..a4a35e9241d02 100644 --- a/pkg/disttask/importinto/job_testkit_test.go +++ b/pkg/disttask/importinto/job_testkit_test.go @@ -52,7 +52,7 @@ func TestGetTaskImportedRows(t *testing.T) { } bytes, err := json.Marshal(taskMeta) require.NoError(t, err) - taskID, err := manager.AddNewGlobalTask(ctx, importinto.TaskKey(111), proto.ImportInto, 1, bytes) + taskID, err := manager.CreateTask(ctx, importinto.TaskKey(111), proto.ImportInto, 1, bytes) require.NoError(t, err) importStepMetas := []*importinto.ImportStepMeta{ { @@ -69,7 +69,7 @@ func TestGetTaskImportedRows(t *testing.T) { for _, m := range importStepMetas { bytes, err := json.Marshal(m) require.NoError(t, err) - require.NoError(t, manager.AddNewSubTask(ctx, taskID, importinto.StepImport, + require.NoError(t, manager.CreateSubTask(ctx, taskID, importinto.StepImport, "", bytes, proto.ImportInto, false)) } rows, err := importinto.GetTaskImportedRows(ctx, 111) @@ -84,7 +84,7 @@ func TestGetTaskImportedRows(t *testing.T) { } bytes, err = json.Marshal(taskMeta) require.NoError(t, err) - taskID, err = manager.AddNewGlobalTask(ctx, importinto.TaskKey(222), proto.ImportInto, 1, bytes) + taskID, err = manager.CreateTask(ctx, importinto.TaskKey(222), proto.ImportInto, 1, bytes) require.NoError(t, err) ingestStepMetas := []*importinto.WriteIngestStepMeta{ { @@ -101,7 +101,7 @@ func TestGetTaskImportedRows(t *testing.T) { for _, m := range ingestStepMetas { bytes, err := json.Marshal(m) require.NoError(t, err) - require.NoError(t, manager.AddNewSubTask(ctx, taskID, importinto.StepWriteAndIngest, + require.NoError(t, manager.CreateSubTask(ctx, taskID, importinto.StepWriteAndIngest, "", bytes, proto.ImportInto, false)) } rows, err = importinto.GetTaskImportedRows(ctx, 222) diff --git a/pkg/disttask/importinto/subtask_executor.go b/pkg/disttask/importinto/subtask_executor.go index 9eb203b2f9514..10900ce372417 100644 --- a/pkg/disttask/importinto/subtask_executor.go +++ b/pkg/disttask/importinto/subtask_executor.go @@ -135,12 +135,12 @@ func verifyChecksum(ctx context.Context, taskMeta *TaskMeta, subtaskMeta *PostPr <-ctx.Done() }) - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return err } - remoteChecksum, err := checksumTable(ctx, globalTaskManager, taskMeta, logger) + remoteChecksum, err := checksumTable(ctx, taskManager, taskMeta, logger) if err != nil { if taskMeta.Plan.Checksum != config.OpLevelOptional { return err diff --git a/pkg/executor/import_into.go b/pkg/executor/import_into.go index 4cc9c3275271a..5d65c9d0044e0 100644 --- a/pkg/executor/import_into.go +++ b/pkg/executor/import_into.go @@ -177,14 +177,14 @@ func (e *ImportIntoExec) Next(ctx context.Context, req *chunk.Chunk) (err error) func (e *ImportIntoExec) fillJobInfo(ctx context.Context, jobID int64, req *chunk.Chunk) error { e.dataFilled = true - // we use globalTaskManager to get job, user might not have the privilege to system tables. - globalTaskManager, err := fstorage.GetTaskManager() + // we use taskManager to get job, user might not have the privilege to system tables. + taskManager, err := fstorage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return err } var info *importer.JobInfo - if err = globalTaskManager.WithNewSession(func(se sessionctx.Context) error { + if err = taskManager.WithNewSession(func(se sessionctx.Context) error { sqlExec := se.(sqlexec.SQLExecutor) var err2 error info, err2 = importer.GetJob(ctx, sqlExec, jobID, e.Ctx().GetSessionVars().User.String(), false) @@ -224,12 +224,12 @@ func (e *ImportIntoExec) doImport(ctx context.Context, se sessionctx.Context, di err := group.Wait() // when user KILL the connection, the ctx will be canceled, we need to cancel the import job. if errors.Cause(err) == context.Canceled { - globalTaskManager, err2 := fstorage.GetTaskManager() + taskManager, err2 := fstorage.GetTaskManager() if err2 != nil { return err2 } // use background, since ctx is canceled already. - return cancelAndWaitImportJob(context.Background(), globalTaskManager, distImporter.JobID()) + return cancelAndWaitImportJob(context.Background(), taskManager, distImporter.JobID()) } if err2 := flushStats(ctx, se, e.importPlan.TableInfo.ID, distImporter.Result(ctx)); err2 != nil { logutil.Logger(ctx).Error("flush stats failed", zap.Error(err2)) @@ -257,12 +257,12 @@ func (e *ImportIntoActionExec) Next(ctx context.Context, _ *chunk.Chunk) (err er hasSuperPriv = pm.RequestVerification(e.Ctx().GetSessionVars().ActiveRoles, "", "", "", mysql.SuperPriv) } // we use sessionCtx from GetTaskManager, user ctx might not have enough privileges. - globalTaskManager, err := fstorage.GetTaskManager() + taskManager, err := fstorage.GetTaskManager() ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return err } - if err = e.checkPrivilegeAndStatus(ctx, globalTaskManager, hasSuperPriv); err != nil { + if err = e.checkPrivilegeAndStatus(ctx, taskManager, hasSuperPriv); err != nil { return err } @@ -271,7 +271,7 @@ func (e *ImportIntoActionExec) Next(ctx context.Context, _ *chunk.Chunk) (err er defer func() { task.End(zap.ErrorLevel, err) }() - return cancelAndWaitImportJob(ctx, globalTaskManager, e.jobID) + return cancelAndWaitImportJob(ctx, taskManager, e.jobID) } func (e *ImportIntoActionExec) checkPrivilegeAndStatus(ctx context.Context, manager *fstorage.TaskManager, hasSuperPriv bool) error { @@ -306,7 +306,7 @@ func flushStats(ctx context.Context, se sessionctx.Context, tableID int64, resul func cancelAndWaitImportJob(ctx context.Context, manager *fstorage.TaskManager, jobID int64) error { if err := manager.WithNewTxn(ctx, func(se sessionctx.Context) error { ctx = util.WithInternalSourceType(ctx, kv.InternalDistTask) - return manager.CancelGlobalTaskByKeySession(ctx, se, importinto.TaskKey(jobID)) + return manager.CancelTaskByKeySession(ctx, se, importinto.TaskKey(jobID)) }); err != nil { return err } diff --git a/pkg/executor/show.go b/pkg/executor/show.go index cbb924464f1f3..a39c438e59f2e 100644 --- a/pkg/executor/show.go +++ b/pkg/executor/show.go @@ -2309,14 +2309,14 @@ func (e *ShowExec) fetchShowImportJobs(ctx context.Context) error { hasSuperPriv = pm.RequestVerification(e.Ctx().GetSessionVars().ActiveRoles, "", "", "", mysql.SuperPriv) } // we use sessionCtx from GetTaskManager, user ctx might not have system table privileges. - globalTaskManager, err := fstorage.GetTaskManager() + taskManager, err := fstorage.GetTaskManager() ctx = kv.WithInternalSourceType(ctx, kv.InternalDistTask) if err != nil { return err } if e.ImportJobID != nil { var info *importer.JobInfo - if err = globalTaskManager.WithNewSession(func(se sessionctx.Context) error { + if err = taskManager.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) var err2 error info, err2 = importer.GetJob(ctx, exec, *e.ImportJobID, e.Ctx().GetSessionVars().User.String(), hasSuperPriv) @@ -2327,7 +2327,7 @@ func (e *ShowExec) fetchShowImportJobs(ctx context.Context) error { return handleImportJobInfo(ctx, info, e.result) } var infos []*importer.JobInfo - if err = globalTaskManager.WithNewSession(func(se sessionctx.Context) error { + if err = taskManager.WithNewSession(func(se sessionctx.Context) error { exec := se.(sqlexec.SQLExecutor) var err2 error infos, err2 = importer.GetAllViewableJobs(ctx, exec, e.Ctx().GetSessionVars().User.String(), hasSuperPriv) diff --git a/pkg/session/bootstrap.go b/pkg/session/bootstrap.go index 6940de8e0cd78..bff40d15c6c8c 100644 --- a/pkg/session/bootstrap.go +++ b/pkg/session/bootstrap.go @@ -575,8 +575,11 @@ const ( type VARCHAR(256) NOT NULL, dispatcher_id VARCHAR(256), state VARCHAR(64) NOT NULL, + priority INT NOT NULL, + create_time TIMESTAMP, start_time TIMESTAMP, state_update_time TIMESTAMP, + end_time TIMESTAMP, meta LONGBLOB, concurrency INT(11), step INT(11), @@ -592,8 +595,11 @@ const ( type VARCHAR(256) NOT NULL, dispatcher_id VARCHAR(256), state VARCHAR(64) NOT NULL, + priority INT NOT NULL, + create_time TIMESTAMP, start_time TIMESTAMP, state_update_time TIMESTAMP, + end_time TIMESTAMP, meta LONGBLOB, concurrency INT(11), step INT(11), @@ -1023,14 +1029,20 @@ const ( // write mDDLTableVersion into `mysql.tidb` table version178 = 178 - // vresion 179 + // version 179 // enlarge `VARIABLE_VALUE` of `mysql.global_variables` from `varchar(1024)` to `varchar(16383)`. version179 = 179 + + // version 180 + // add priority/create_time/end_time to `mysql.tidb_global_task`/`mysql.tidb_global_task_history` + // add concurrency/priority/create_time/end_time to `mysql.tidb_background_subtask`/`mysql.tidb_background_subtask_history` + // add idx_exec_id(exec_id) to `mysql.tidb_background_subtask` + version180 = 180 ) // currentBootstrapVersion is defined as a variable, so we can modify its value for testing. // please make sure this is the largest version -var currentBootstrapVersion int64 = version179 +var currentBootstrapVersion int64 = version180 // DDL owner key's expired time is ManagerSessionTTL seconds, we should wait the time and give more time to have a chance to finish it. var internalSQLTimeout = owner.ManagerSessionTTL + 15 @@ -1185,6 +1197,7 @@ var ( upgradeToVer177, upgradeToVer178, upgradeToVer179, + upgradeToVer180, } ) @@ -2899,6 +2912,27 @@ func upgradeToVer179(s sessiontypes.Session, ver int64) { doReentrantDDL(s, "ALTER TABLE mysql.global_variables MODIFY COLUMN `VARIABLE_VALUE` varchar(16383)") } +func upgradeToVer180(s sessiontypes.Session, ver int64) { + if ver >= version180 { + return + } + doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task ADD COLUMN `priority` INT NOT NULL AFTER `state`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task ADD COLUMN `create_time` TIMESTAMP AFTER `priority`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task ADD COLUMN `end_time` TIMESTAMP AFTER `state_update_time`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task_history ADD COLUMN `priority` INT NOT NULL AFTER `state`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task_history ADD COLUMN `create_time` TIMESTAMP AFTER `priority`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_global_task_history ADD COLUMN `end_time` TIMESTAMP AFTER `state_update_time`", infoschema.ErrColumnExists) + + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD COLUMN `concurrency` INT AFTER `checkpoint`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD COLUMN `create_time` TIMESTAMP AFTER `concurrency`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD COLUMN `end_time` TIMESTAMP AFTER `state_update_time`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask_history ADD COLUMN `concurrency` INT AFTER `checkpoint`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask_history ADD COLUMN `create_time` TIMESTAMP AFTER `concurrency`", infoschema.ErrColumnExists) + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask_history ADD COLUMN `end_time` TIMESTAMP AFTER `state_update_time`", infoschema.ErrColumnExists) + + doReentrantDDL(s, "ALTER TABLE mysql.tidb_background_subtask ADD INDEX idx_exec_id(exec_id)", dbterror.ErrDupKeyName) +} + func writeOOMAction(s sessiontypes.Session) { comment := "oom-action is `log` by default in v3.0.x, `cancel` by default in v4.0.11+" mustExecute(s, `INSERT HIGH_PRIORITY INTO %n.%n VALUES (%?, %?, %?) ON DUPLICATE KEY UPDATE VARIABLE_VALUE= %?`, diff --git a/tests/integrationtest/r/session/bootstrap_upgrade.result b/tests/integrationtest/r/session/bootstrap_upgrade.result index 607399d02891b..a6986368c06d7 100644 --- a/tests/integrationtest/r/session/bootstrap_upgrade.result +++ b/tests/integrationtest/r/session/bootstrap_upgrade.result @@ -36,8 +36,11 @@ tidb_background_subtask_history CREATE TABLE `tidb_background_subtask_history` ( `exec_expired` timestamp NULL DEFAULT NULL, `state` varchar(64) NOT NULL, `checkpoint` longblob NOT NULL, + `concurrency` int(11) DEFAULT NULL, + `create_time` timestamp NULL DEFAULT NULL, `start_time` bigint(20) DEFAULT NULL, `state_update_time` bigint(20) DEFAULT NULL, + `end_time` timestamp NULL DEFAULT NULL, `meta` longblob DEFAULT NULL, `error` blob DEFAULT NULL, `summary` json DEFAULT NULL, @@ -48,7 +51,7 @@ tidb_background_subtask_history CREATE TABLE `tidb_background_subtask_history` ( insert into tidb_background_subtask(id, state, checkpoint) values (1, 0, ""); insert into tidb_background_subtask_history select * from tidb_background_subtask; select * from tidb_background_subtask_history; -id step namespace task_key ddl_physical_tid type exec_id exec_expired state checkpoint start_time state_update_time meta error summary -1 NULL NULL NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL +id step namespace task_key ddl_physical_tid type exec_id exec_expired state checkpoint concurrency create_time start_time state_update_time end_time meta error summary +1 NULL NULL NULL NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL truncate table tidb_background_subtask; truncate table tidb_background_subtask_history; diff --git a/tests/realtikvtest/importintotest/import_into_test.go b/tests/realtikvtest/importintotest/import_into_test.go index 20ac3c9a1aaac..17b0f0aa00a24 100644 --- a/tests/realtikvtest/importintotest/import_into_test.go +++ b/tests/realtikvtest/importintotest/import_into_test.go @@ -775,23 +775,23 @@ func (s *mockGCSSuite) TestColumnsAndUserVars() { serverInfo, err := infosync.GetServerInfo() s.NoError(err) for _, st := range subtasks { - s.Equal(net.JoinHostPort(serverInfo.IP, strconv.Itoa(int(serverInfo.Port))), st.SchedulerID) + s.Equal(net.JoinHostPort(serverInfo.IP, strconv.Itoa(int(serverInfo.Port))), st.ExecID) } } func (s *mockGCSSuite) checkTaskMetaRedacted(jobID int64) { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() s.NoError(err) taskKey := importinto.TaskKey(jobID) s.NoError(err) ctx := context.Background() ctx = util.WithInternalSourceType(ctx, "taskManager") - globalTask, err2 := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err2 := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) s.NoError(err2) - s.Regexp(`[?&]access-key=xxxxxx`, string(globalTask.Meta)) - s.Contains(string(globalTask.Meta), "secret-access-key=xxxxxx") - s.NotContains(string(globalTask.Meta), "aaaaaa") - s.NotContains(string(globalTask.Meta), "bbbbbb") + s.Regexp(`[?&]access-key=xxxxxx`, string(task.Meta)) + s.Contains(string(task.Meta), "secret-access-key=xxxxxx") + s.NotContains(string(task.Meta), "aaaaaa") + s.NotContains(string(task.Meta), "bbbbbb") } func (s *mockGCSSuite) TestImportMode() { diff --git a/tests/realtikvtest/importintotest/job_test.go b/tests/realtikvtest/importintotest/job_test.go index ed7d9309fad32..717bc449812f5 100644 --- a/tests/realtikvtest/importintotest/job_test.go +++ b/tests/realtikvtest/importintotest/job_test.go @@ -401,12 +401,12 @@ func (s *mockGCSSuite) TestCancelJob() { s.ErrorIs(err, exeerrors.ErrLoadDataJobNotFound) getTask := func(jobID int64) *proto.Task { - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() s.NoError(err) taskKey := importinto.TaskKey(jobID) - globalTask, err := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) s.NoError(err) - return globalTask + return task } // cancel a running job created by self @@ -501,14 +501,14 @@ func (s *mockGCSSuite) TestCancelJob() { ErrorMessage: "cancelled by user", } s.compareJobInfoWithoutTime(jobInfo, rows2[0]) - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() s.NoError(err) taskKey := importinto.TaskKey(int64(jobID2)) s.NoError(err) s.Require().Eventually(func() bool { - globalTask, err2 := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task2, err2 := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) s.NoError(err2) - subtasks, err2 := globalTaskManager.GetSubtasksForImportInto(ctx, globalTask.ID, importinto.StepPostProcess) + subtasks, err2 := taskManager.GetSubtasksForImportInto(ctx, task2.ID, importinto.StepPostProcess) s.NoError(err2) s.Len(subtasks, 2) // framework will generate a subtask when canceling var cancelled bool @@ -518,7 +518,7 @@ func (s *mockGCSSuite) TestCancelJob() { break } } - return globalTask.State == proto.TaskStateReverted && cancelled + return task2.State == proto.TaskStateReverted && cancelled }, maxWaitTime, 1*time.Second) // cancel a pending job created by test_cancel_job2 using root @@ -644,13 +644,13 @@ func (s *mockGCSSuite) TestKillBeforeFinish() { rows := s.tk.MustQuery(fmt.Sprintf("show import job %d", jobID)).Rows() s.Len(rows, 1) s.Equal("cancelled", rows[0][5]) - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() s.NoError(err) taskKey := importinto.TaskKey(jobID) s.NoError(err) s.Require().Eventually(func() bool { - globalTask, err2 := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err2 := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) s.NoError(err2) - return globalTask.State == proto.TaskStateReverted + return task.State == proto.TaskStateReverted }, maxWaitTime, 1*time.Second) } diff --git a/tests/realtikvtest/importintotest4/global_sort_test.go b/tests/realtikvtest/importintotest4/global_sort_test.go index b84b3f9929bf9..88be1e6e59aa7 100644 --- a/tests/realtikvtest/importintotest4/global_sort_test.go +++ b/tests/realtikvtest/importintotest4/global_sort_test.go @@ -87,13 +87,13 @@ func (s *mockGCSSuite) TestGlobalSortBasic() { redactedSortStorageURI := fmt.Sprintf("gs://sorted/import?endpoint=%s&access-key=xxxxxx&secret-access-key=xxxxxx", gcsEndpoint) urlEqual(s.T(), redactedSortStorageURI, jobInfo.Parameters.Options["cloud_storage_uri"].(string)) s.Equal(uint64(6), jobInfo.Summary.ImportedRows) - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() s.NoError(err) taskKey := importinto.TaskKey(int64(jobID)) - globalTask, err2 := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err2 := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) s.NoError(err2) taskMeta := importinto.TaskMeta{} - s.NoError(json.Unmarshal(globalTask.Meta, &taskMeta)) + s.NoError(json.Unmarshal(task.Meta, &taskMeta)) urlEqual(s.T(), redactedSortStorageURI, taskMeta.Plan.CloudStorageURI) // merge-sort data kv @@ -115,9 +115,9 @@ func (s *mockGCSSuite) TestGlobalSortBasic() { jobID, err = strconv.Atoi(result[0][0].(string)) s.NoError(err) s.Eventually(func() bool { - globalTask, err2 = globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, importinto.TaskKey(int64(jobID))) + task, err2 = taskManager.GetTaskByKeyWithHistory(ctx, importinto.TaskKey(int64(jobID))) s.NoError(err2) - return globalTask.State == "failed" + return task.State == "failed" }, 30*time.Second, 300*time.Millisecond) // check all sorted data cleaned up <-dispatcher.WaitCleanUpFinished diff --git a/tests/realtikvtest/importintotest4/split_file_test.go b/tests/realtikvtest/importintotest4/split_file_test.go index 6d8a5bf7e4665..7f8484d6148ba 100644 --- a/tests/realtikvtest/importintotest4/split_file_test.go +++ b/tests/realtikvtest/importintotest4/split_file_test.go @@ -58,14 +58,14 @@ func (s *mockGCSSuite) TestSplitFile() { s.Len(result, 1) jobID, err := strconv.Atoi(result[0][0].(string)) s.NoError(err) - globalTaskManager, err := storage.GetTaskManager() + taskManager, err := storage.GetTaskManager() s.NoError(err) taskKey := importinto.TaskKey(int64(jobID)) s.NoError(err) - globalTask, err2 := globalTaskManager.GetGlobalTaskByKeyWithHistory(ctx, taskKey) + task, err2 := taskManager.GetTaskByKeyWithHistory(ctx, taskKey) s.NoError(err2) - subtasks, err2 := globalTaskManager.GetSubtasksForImportInto(ctx, globalTask.ID, importinto.StepImport) + subtasks, err2 := taskManager.GetSubtasksForImportInto(ctx, task.ID, importinto.StepImport) s.NoError(err2) s.Len(subtasks, 3) s.tk.MustQuery("select * from t").Sort().Check(testkit.Rows(allData...))