Skip to content

Commit

Permalink
test: switch some failpoints to InjectCall (#53588)
Browse files Browse the repository at this point in the history
ref #49008
  • Loading branch information
D3Hunter authored May 28, 2024
1 parent f7cea6c commit d49a654
Show file tree
Hide file tree
Showing 44 changed files with 353 additions and 346 deletions.
13 changes: 2 additions & 11 deletions pkg/ddl/backfilling_operators.go
Original file line number Diff line number Diff line change
Expand Up @@ -444,9 +444,6 @@ func (w *tableScanWorker) Close() {
}
}

// OperatorCallBackForTest is used for test to mock scan record error.
var OperatorCallBackForTest func()

func (w *tableScanWorker) scanRecords(task TableScanTask, sender func(IndexRecordChunk)) {
logutil.Logger(w.ctx).Info("start a table scan task",
zap.Int("id", task.ID), zap.Stringer("task", task))
Expand All @@ -456,9 +453,7 @@ func (w *tableScanWorker) scanRecords(task TableScanTask, sender func(IndexRecor
failpoint.Inject("mockScanRecordError", func(_ failpoint.Value) {
failpoint.Return(errors.New("mock scan record error"))
})
failpoint.Inject("scanRecordExec", func(_ failpoint.Value) {
OperatorCallBackForTest()
})
failpoint.InjectCall("scanRecordExec")
rs, err := buildTableScan(w.ctx, w.copCtx.GetBase(), startTS, task.Start, task.End)
if err != nil {
return err
Expand Down Expand Up @@ -749,11 +744,7 @@ func (w *indexIngestBaseWorker) WriteChunk(rs *IndexRecordChunk) (count int, nex
failpoint.Inject("mockWriteLocalError", func(_ failpoint.Value) {
failpoint.Return(0, nil, errors.New("mock write local error"))
})
failpoint.Inject("writeLocalExec", func(_ failpoint.Value) {
if rs.Done {
OperatorCallBackForTest()
}
})
failpoint.InjectCall("writeLocalExec", rs.Done)

oprStartTime := time.Now()
vars := w.se.GetSessionVars()
Expand Down
10 changes: 1 addition & 9 deletions pkg/ddl/backfilling_read_index.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,16 +149,8 @@ func (r *readIndexExecutor) Cleanup(ctx context.Context) error {
return nil
}

// MockDMLExecutionAddIndexSubTaskFinish is used to mock DML execution during distributed add index.
var MockDMLExecutionAddIndexSubTaskFinish func()

func (r *readIndexExecutor) OnFinished(ctx context.Context, subtask *proto.Subtask) error {
failpoint.Inject("mockDMLExecutionAddIndexSubTaskFinish", func(val failpoint.Value) {
//nolint:forcetypeassert
if val.(bool) {
MockDMLExecutionAddIndexSubTaskFinish()
}
})
failpoint.InjectCall("mockDMLExecutionAddIndexSubTaskFinish")
if len(r.cloudStorageURI) == 0 {
return nil
}
Expand Down
18 changes: 2 additions & 16 deletions pkg/ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -2019,15 +2019,6 @@ func checkDuplicateForUniqueIndex(ctx context.Context, t table.Table, reorgInfo
return nil
}

// MockDMLExecutionOnTaskFinished is used to mock DML execution when tasks finished.
var MockDMLExecutionOnTaskFinished func()

// MockDMLExecutionOnDDLPaused is used to mock DML execution when ddl job paused.
var MockDMLExecutionOnDDLPaused func()

// TestSyncChan is used to sync the test.
var TestSyncChan = make(chan struct{})

func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
if reorgInfo.mergingTmpIdx {
return errors.New("do not support merge index")
Expand Down Expand Up @@ -2115,9 +2106,7 @@ func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
g.Go(func() error {
defer close(done)
err := submitAndWaitTask(ctx, taskKey, taskType, concurrency, reorgInfo.ReorgMeta.TargetScope, metaData)
failpoint.Inject("pauseAfterDistTaskFinished", func() {
MockDMLExecutionOnTaskFinished()
})
failpoint.InjectCall("pauseAfterDistTaskFinished")
if err := w.isReorgRunnable(reorgInfo.Job.ID, true); err != nil {
if dbterror.ErrPausedDDLJob.Equal(err) {
logutil.DDLLogger().Warn("job paused by user", zap.Error(err))
Expand Down Expand Up @@ -2145,10 +2134,7 @@ func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
logutil.DDLLogger().Error("pause task error", zap.String("task_key", taskKey), zap.Error(err))
continue
}
failpoint.Inject("syncDDLTaskPause", func() {
// make sure the task is paused.
TestSyncChan <- struct{}{}
})
failpoint.InjectCall("syncDDLTaskPause")
}
if !dbterror.ErrCancelledDDLJob.Equal(err) {
return errors.Trace(err)
Expand Down
1 change: 1 addition & 0 deletions pkg/disttask/framework/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@ go_test(
"//pkg/disttask/framework/proto",
"//pkg/disttask/framework/storage",
"//pkg/testkit",
"//pkg/testkit/testfailpoint",
"//pkg/util/backoff",
"@com_github_ngaut_pools//:pools",
"@com_github_pingcap_errors//:errors",
Expand Down
3 changes: 2 additions & 1 deletion pkg/disttask/framework/handle/handle_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,13 +28,14 @@ import (
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
"github.com/pingcap/tidb/pkg/disttask/framework/storage"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/util/backoff"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/util"
)

func TestHandle(t *testing.T) {
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/util/cpu/mockNumCpu", "return(8)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/util/cpu/mockNumCpu", "return(8)")

ctx := util.WithInternalSourceType(context.Background(), "handle_test")

Expand Down
1 change: 1 addition & 0 deletions pkg/disttask/framework/integrationtests/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,7 @@ go_test(
"//pkg/session",
"//pkg/store/driver",
"//pkg/testkit",
"//pkg/testkit/testfailpoint",
"//pkg/testkit/testsetup",
"//pkg/util",
"@com_github_pingcap_failpoint//:failpoint",
Expand Down
3 changes: 2 additions & 1 deletion pkg/disttask/framework/integrationtests/bench_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ import (
"github.com/pingcap/tidb/pkg/session"
"github.com/pingcap/tidb/pkg/store/driver"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/stretchr/testify/require"
"go.opencensus.io/stats/view"
"go.uber.org/mock/gomock"
Expand Down Expand Up @@ -106,7 +107,7 @@ func BenchmarkSchedulerOverhead(b *testing.B) {
}

func prepareForBenchTest(b *testing.B) {
testkit.EnableFailPoint(b, "github.com/pingcap/tidb/pkg/domain/MockDisableDistTask", "return(true)")
testfailpoint.Enable(b, "github.com/pingcap/tidb/pkg/domain/MockDisableDistTask", "return(true)")

var d driver.TiKVDriver
var err error
Expand Down
6 changes: 3 additions & 3 deletions pkg/disttask/framework/integrationtests/framework_ha_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
"github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/util"
"github.com/stretchr/testify/require"
)
Expand All @@ -36,7 +36,7 @@ func submitTaskAndCheckSuccessForHA(ctx context.Context, t *testing.T, taskKey s
}

func TestHANodeRandomShutdown(t *testing.T) {
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBShutdown", "return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBShutdown", "return()")
c := testutil.NewDXFContextWithRandomNodes(t, 4, 15)
testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockHATestSchedulerExt(c.MockCtrl), c.TestContext, nil)

Expand All @@ -55,7 +55,7 @@ func TestHANodeRandomShutdown(t *testing.T) {
}

func TestHARandomShutdownInDifferentStep(t *testing.T) {
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBShutdown", "return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockTiDBShutdown", "return()")
c := testutil.NewDXFContextWithRandomNodes(t, 6, 15)

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockHATestSchedulerExt(c.MockCtrl), c.TestContext, nil)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
"github.com/pingcap/tidb/pkg/disttask/framework/storage"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/stretchr/testify/require"
)

Expand All @@ -50,7 +50,7 @@ func TestFrameworkPauseAndResume(t *testing.T) {

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
// 1. schedule and pause one running task.
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pauseTaskAfterRefreshTask", "2*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pauseTaskAfterRefreshTask", "2*return(true)")
task1 := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStatePaused, task1.State)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pauseTaskAfterRefreshTask"))
Expand All @@ -67,7 +67,7 @@ func TestFrameworkPauseAndResume(t *testing.T) {
require.Empty(t, errs)

// 2. pause pending task.
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pausePendingTask", "2*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pausePendingTask", "2*return(true)")
task2 := testutil.SubmitAndWaitTask(c.Ctx, t, "key2", "", 1)
require.Equal(t, proto.TaskStatePaused, task2.State)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/pausePendingTask"))
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,14 +19,14 @@ import (

"github.com/pingcap/tidb/pkg/disttask/framework/proto"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/stretchr/testify/require"
)

func TestFrameworkRollback(t *testing.T) {
c := testutil.NewTestDXFContext(t, 2, 16, true)
testutil.RegisterRollbackTaskMeta(t, c.MockCtrl, testutil.GetMockRollbackSchedulerExt(c.MockCtrl), c.TestContext)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelTaskAfterRefreshTask", "2*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelTaskAfterRefreshTask", "2*return(true)")

task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/pkg/disttask/framework/storage"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/stretchr/testify/require"
"go.uber.org/mock/gomock"
"golang.org/x/exp/rand"
Expand Down Expand Up @@ -80,7 +81,7 @@ func TestScopeBasic(t *testing.T) {
tk.MustQuery("select @@global.tidb_service_scope").Check(testkit.Rows("background"))
tk.MustQuery("select @@tidb_service_scope").Check(testkit.Rows("background"))

testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh", "1*return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh", "1*return()")
<-scheduler.TestRefreshedChan
taskID = submitTaskAndCheckSuccessForScope(c.Ctx, t, "😊", nodeCnt, "background", c.TestContext)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh"))
Expand All @@ -93,7 +94,7 @@ func TestScopeBasic(t *testing.T) {
// 3. 2 "background" role.
tk.MustExec("update mysql.dist_framework_meta set role = \"background\" where host = \":4001\"")
time.Sleep(5 * time.Second)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh", "1*return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh", "1*return()")
<-scheduler.TestRefreshedChan
taskID = submitTaskAndCheckSuccessForScope(c.Ctx, t, "😆", nodeCnt, "background", c.TestContext)
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh"))
Expand Down Expand Up @@ -145,7 +146,7 @@ func runTargetScopeCase(t *testing.T, c *testutil.TestDXFContext, tk *testkit.Te
for i := 0; i < len(testCase.nodeScopes); i++ {
tk.MustExec(fmt.Sprintf("update mysql.dist_framework_meta set role = \"%s\" where host = \"%s\"", testCase.nodeScopes[i], c.GetNodeIDByIdx(i)))
}
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh", "3*return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/syncRefresh", "3*return()")
<-scheduler.TestRefreshedChan
<-scheduler.TestRefreshedChan
<-scheduler.TestRefreshedChan
Expand Down
37 changes: 23 additions & 14 deletions pkg/disttask/framework/integrationtests/framework_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,14 +18,17 @@ import (
"context"
"fmt"
"math/rand"
"sync"
"testing"
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/disttask/framework/proto"
"github.com/pingcap/tidb/pkg/disttask/framework/scheduler"
"github.com/pingcap/tidb/pkg/disttask/framework/storage"
"github.com/pingcap/tidb/pkg/disttask/framework/testutil"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/pkg/util"
"github.com/stretchr/testify/require"
)
Expand Down Expand Up @@ -126,7 +129,7 @@ func TestFrameworkCancelTask(t *testing.T) {

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)

testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunCancel", "1*return(1)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunCancel", "1*return(1)")
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}
Expand All @@ -135,32 +138,38 @@ func TestFrameworkSubTaskFailed(t *testing.T) {
c := testutil.NewTestDXFContext(t, 1, 16, true)

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunErr", "1*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockExecutorRunErr", "1*return(true)")
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}

func TestFrameworkSubTaskInitEnvFailed(t *testing.T) {
c := testutil.NewTestDXFContext(t, 1, 16, true)
testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockExecSubtaskInitEnvErr", "return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/mockExecSubtaskInitEnvErr", "return()")
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}

func TestOwnerChangeWhenSchedule(t *testing.T) {
c := testutil.NewTestDXFContext(t, 3, 16, true)
testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
scheduler.MockOwnerChange = func() {
c.AsyncChangeOwner()
}
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockOwnerChange", "1*return(true)")
var once sync.Once
require.NoError(t, failpoint.EnableCall("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockOwnerChange", func() {
once.Do(func() {
c.AsyncChangeOwner()
time.Sleep(time.Second)
})
}))
t.Cleanup(func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockOwnerChange"))
})
submitTaskAndCheckSuccessForBasic(c.Ctx, t, "😊", c.TestContext)
}

func TestGC(t *testing.T) {
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/storage/subtaskHistoryKeepSeconds", "return(1)")
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/historySubtaskTableGcInterval", "return(1)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/storage/subtaskHistoryKeepSeconds", "return(1)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/historySubtaskTableGcInterval", "return(1)")
c := testutil.NewTestDXFContext(t, 3, 16, true)

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
Expand Down Expand Up @@ -194,7 +203,7 @@ func TestFrameworkSubtaskFinishedCancel(t *testing.T) {
c := testutil.NewTestDXFContext(t, 3, 16, true)

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockSubtaskFinishedCancel", "1*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockSubtaskFinishedCancel", "1*return(true)")
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}
Expand All @@ -203,7 +212,7 @@ func TestFrameworkRunSubtaskCancel(t *testing.T) {
c := testutil.NewTestDXFContext(t, 3, 16, true)

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockRunSubtaskCancel", "1*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/taskexecutor/MockRunSubtaskCancel", "1*return(true)")
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}
Expand All @@ -216,7 +225,7 @@ func TestFrameworkCleanUpRoutine(t *testing.T) {
scheduler.DefaultCleanUpInterval = 500 * time.Millisecond
c := testutil.NewTestDXFContext(t, 3, 16, true)
testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished", "return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/WaitCleanUpFinished", "return()")

// normal
submitTaskAndCheckSuccessForBasic(c.Ctx, t, "key1", c.TestContext)
Expand All @@ -231,7 +240,7 @@ func TestFrameworkCleanUpRoutine(t *testing.T) {
require.NotEmpty(t, subtasks)

// transfer err
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockTransferErr", "1*return()")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/mockTransferErr", "1*return()")
submitTaskAndCheckSuccessForBasic(c.Ctx, t, "key2", c.TestContext)
<-scheduler.WaitCleanUpFinished
mgr, err = storage.GetTaskManager()
Expand All @@ -248,7 +257,7 @@ func TestTaskCancelledBeforeUpdateTask(t *testing.T) {
c := testutil.NewTestDXFContext(t, 1, 16, true)

testutil.RegisterTaskMeta(t, c.MockCtrl, testutil.GetMockBasicSchedulerExt(c.MockCtrl), c.TestContext, nil)
testkit.EnableFailPoint(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelBeforeUpdateTask", "1*return(true)")
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/disttask/framework/scheduler/cancelBeforeUpdateTask", "1*return(true)")
task := testutil.SubmitAndWaitTask(c.Ctx, t, "key1", "", 1)
require.Equal(t, proto.TaskStateReverted, task.State)
}
1 change: 1 addition & 0 deletions pkg/disttask/framework/scheduler/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -69,6 +69,7 @@ go_test(
"//pkg/kv",
"//pkg/sessionctx",
"//pkg/testkit",
"//pkg/testkit/testfailpoint",
"//pkg/testkit/testsetup",
"//pkg/util/cpu",
"//pkg/util/disttask",
Expand Down
Loading

0 comments on commit d49a654

Please sign in to comment.