Skip to content

Commit e49a151

Browse files
authored
ddl: check context done in isReorgRunnable function (#57813) (#57820)
close #57325
1 parent 43b21d4 commit e49a151

File tree

7 files changed

+78
-72
lines changed

7 files changed

+78
-72
lines changed

pkg/ddl/backfilling.go

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -327,7 +327,7 @@ func (w *backfillWorker) handleBackfillTask(d *ddlCtx, task *reorgBackfillTask,
327327
// we will never cancel the job once there is panic in bf.BackfillData.
328328
// Because reorgRecordTask may run a long time,
329329
// we should check whether this ddl job is still runnable.
330-
err := d.isReorgRunnable(jobID, false)
330+
err := d.isReorgRunnable(d.ctx, false)
331331
if err != nil {
332332
result.err = err
333333
return result
@@ -677,8 +677,7 @@ func (dc *ddlCtx) runAddIndexInLocalIngestMode(
677677
t table.PhysicalTable,
678678
reorgInfo *reorgInfo,
679679
) error {
680-
// TODO(tangenta): support adjust worker count dynamically.
681-
if err := dc.isReorgRunnable(reorgInfo.Job.ID, false); err != nil {
680+
if err := dc.isReorgRunnable(ctx, false); err != nil {
682681
return errors.Trace(err)
683682
}
684683
job := reorgInfo.Job
@@ -921,7 +920,7 @@ func (dc *ddlCtx) writePhysicalTableRecord(
921920
) (err error) {
922921
startKey, endKey := reorgInfo.StartKey, reorgInfo.EndKey
923922

924-
if err := dc.isReorgRunnable(reorgInfo.Job.ID, false); err != nil {
923+
if err := dc.isReorgRunnable(ctx, false); err != nil {
925924
return errors.Trace(err)
926925
}
927926
defer func() {

pkg/ddl/ddl_test.go

Lines changed: 0 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -52,11 +52,6 @@ type DDLForTest interface {
5252
RemoveReorgCtx(id int64)
5353
}
5454

55-
// IsReorgCanceled exports for testing.
56-
func (rc *reorgCtx) IsReorgCanceled() bool {
57-
return rc.isReorgCanceled()
58-
}
59-
6055
// NewReorgCtx exports for testing.
6156
func (d *ddl) NewReorgCtx(jobID int64, rowCount int64) *reorgCtx {
6257
return d.newReorgCtx(jobID, rowCount)

pkg/ddl/index.go

Lines changed: 5 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -2388,7 +2388,7 @@ func (w *worker) addTableIndex(
23882388
// TODO: Support typeAddIndexMergeTmpWorker.
23892389
if reorgInfo.ReorgMeta.IsDistReorg && !reorgInfo.mergingTmpIdx {
23902390
if reorgInfo.ReorgMeta.ReorgTp == model.ReorgTypeLitMerge {
2391-
err := w.executeDistTask(t, reorgInfo)
2391+
err := w.executeDistTask(ctx, t, reorgInfo)
23922392
if err != nil {
23932393
return err
23942394
}
@@ -2469,7 +2469,7 @@ func checkDuplicateForUniqueIndex(ctx context.Context, t table.Table, reorgInfo
24692469
return nil
24702470
}
24712471

2472-
func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
2472+
func (w *worker) executeDistTask(stepCtx context.Context, t table.Table, reorgInfo *reorgInfo) error {
24732473
if reorgInfo.mergingTmpIdx {
24742474
return errors.New("do not support merge index")
24752475
}
@@ -2520,7 +2520,7 @@ func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
25202520
return err
25212521
}
25222522
err = handle.WaitTaskDoneOrPaused(ctx, task.ID)
2523-
if err := w.isReorgRunnable(reorgInfo.Job.ID, true); err != nil {
2523+
if err := w.isReorgRunnable(stepCtx, true); err != nil {
25242524
if dbterror.ErrPausedDDLJob.Equal(err) {
25252525
logutil.DDLLogger().Warn("job paused by user", zap.Error(err))
25262526
return dbterror.ErrPausedDDLJob.GenWithStackByArgs(reorgInfo.Job.ID)
@@ -2557,7 +2557,7 @@ func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
25572557
defer close(done)
25582558
err := submitAndWaitTask(ctx, taskKey, taskType, concurrency, reorgInfo.ReorgMeta.TargetScope, metaData)
25592559
failpoint.InjectCall("pauseAfterDistTaskFinished")
2560-
if err := w.isReorgRunnable(reorgInfo.Job.ID, true); err != nil {
2560+
if err := w.isReorgRunnable(stepCtx, true); err != nil {
25612561
if dbterror.ErrPausedDDLJob.Equal(err) {
25622562
logutil.DDLLogger().Warn("job paused by user", zap.Error(err))
25632563
return dbterror.ErrPausedDDLJob.GenWithStackByArgs(reorgInfo.Job.ID)
@@ -2578,7 +2578,7 @@ func (w *worker) executeDistTask(t table.Table, reorgInfo *reorgInfo) error {
25782578
w.updateDistTaskRowCount(taskKey, reorgInfo.Job.ID)
25792579
return nil
25802580
case <-checkFinishTk.C:
2581-
if err = w.isReorgRunnable(reorgInfo.Job.ID, true); err != nil {
2581+
if err = w.isReorgRunnable(stepCtx, true); err != nil {
25822582
if dbterror.ErrPausedDDLJob.Equal(err) {
25832583
if err = handle.PauseTask(w.workCtx, taskKey); err != nil {
25842584
logutil.DDLLogger().Error("pause task error", zap.String("task_key", taskKey), zap.Error(err))

pkg/ddl/job_worker.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -859,7 +859,7 @@ func (w *worker) runOneJobStep(
859859
logutil.DDLLogger().Info("job is paused",
860860
zap.Int64("job_id", job.ID),
861861
zap.Stringer("state", latestJob.State))
862-
cancelStep(dbterror.ErrPausedDDLJob)
862+
cancelStep(dbterror.ErrPausedDDLJob.FastGenByArgs(job.ID))
863863
return
864864
case model.JobStateDone, model.JobStateSynced:
865865
return

pkg/ddl/job_worker_test.go

Lines changed: 0 additions & 24 deletions
Original file line numberDiff line numberDiff line change
@@ -252,27 +252,3 @@ func TestJobNeedGC(t *testing.T) {
252252
}}}
253253
require.True(t, ddl.JobNeedGC(job))
254254
}
255-
256-
func TestUsingReorgCtx(t *testing.T) {
257-
_, domain := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease)
258-
d := domain.DDL()
259-
260-
wg := util.WaitGroupWrapper{}
261-
wg.Run(func() {
262-
jobID := int64(1)
263-
for i := 0; i < 500; i++ {
264-
d.(ddl.DDLForTest).NewReorgCtx(jobID, 0)
265-
d.(ddl.DDLForTest).GetReorgCtx(jobID).IsReorgCanceled()
266-
d.(ddl.DDLForTest).RemoveReorgCtx(jobID)
267-
}
268-
})
269-
wg.Run(func() {
270-
jobID := int64(1)
271-
for i := 0; i < 500; i++ {
272-
d.(ddl.DDLForTest).NewReorgCtx(jobID, 0)
273-
d.(ddl.DDLForTest).GetReorgCtx(jobID).IsReorgCanceled()
274-
d.(ddl.DDLForTest).RemoveReorgCtx(jobID)
275-
}
276-
})
277-
wg.Wait()
278-
}

pkg/ddl/reorg.go

Lines changed: 3 additions & 32 deletions
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,6 @@ type reorgCtx struct {
7474
doneCh chan reorgFnResult
7575
// rowCount is used to simulate a job's row count.
7676
rowCount int64
77-
jobState model.JobState
7877

7978
mu struct {
8079
sync.Mutex
@@ -275,20 +274,6 @@ func reorgTimeZoneWithTzLoc(tzLoc *model.TimeZoneLocation) (*time.Location, erro
275274
return tzLoc.GetLocation()
276275
}
277276

278-
func (rc *reorgCtx) notifyJobState(state model.JobState) {
279-
atomic.StoreInt32((*int32)(&rc.jobState), int32(state))
280-
}
281-
282-
func (rc *reorgCtx) isReorgCanceled() bool {
283-
s := atomic.LoadInt32((*int32)(&rc.jobState))
284-
return int32(model.JobStateCancelled) == s || int32(model.JobStateCancelling) == s
285-
}
286-
287-
func (rc *reorgCtx) isReorgPaused() bool {
288-
s := atomic.LoadInt32((*int32)(&rc.jobState))
289-
return int32(model.JobStatePaused) == s || int32(model.JobStatePausing) == s
290-
}
291-
292277
func (rc *reorgCtx) setRowCount(count int64) {
293278
atomic.StoreInt64(&rc.rowCount, count)
294279
}
@@ -566,28 +551,14 @@ func getTableTotalCount(w *worker, tblInfo *model.TableInfo) int64 {
566551
return rows[0].GetInt64(0)
567552
}
568553

569-
func (dc *ddlCtx) isReorgCancelled(jobID int64) bool {
570-
return dc.getReorgCtx(jobID).isReorgCanceled()
571-
}
572-
func (dc *ddlCtx) isReorgPaused(jobID int64) bool {
573-
return dc.getReorgCtx(jobID).isReorgPaused()
574-
}
575-
576-
func (dc *ddlCtx) isReorgRunnable(jobID int64, isDistReorg bool) error {
554+
func (dc *ddlCtx) isReorgRunnable(ctx context.Context, isDistReorg bool) error {
577555
if dc.ctx.Err() != nil {
578556
// Worker is closed. So it can't do the reorganization.
579557
return dbterror.ErrInvalidWorker.GenWithStack("worker is closed")
580558
}
581559

582-
// TODO(lance6716): check ctx.Err?
583-
if dc.isReorgCancelled(jobID) {
584-
// Job is cancelled. So it can't be done.
585-
return dbterror.ErrCancelledDDLJob
586-
}
587-
588-
if dc.isReorgPaused(jobID) {
589-
logutil.DDLLogger().Warn("job paused by user", zap.String("ID", dc.uuid))
590-
return dbterror.ErrPausedDDLJob.GenWithStackByArgs(jobID)
560+
if ctx.Err() != nil {
561+
return context.Cause(ctx)
591562
}
592563

593564
// If isDistReorg is true, we needn't check if it is owner.

tests/realtikvtest/addindextest1/disttask_test.go

Lines changed: 66 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -17,9 +17,11 @@ package addindextest
1717
import (
1818
"context"
1919
"fmt"
20+
"strings"
2021
"sync"
2122
"sync/atomic"
2223
"testing"
24+
"time"
2325

2426
"github.com/pingcap/failpoint"
2527
"github.com/pingcap/tidb/pkg/config"
@@ -109,7 +111,7 @@ func TestAddIndexDistBasic(t *testing.T) {
109111
tk.MustExec(`set global tidb_enable_dist_task=0;`)
110112
}
111113

112-
func TestAddIndexDistCancel(t *testing.T) {
114+
func TestAddIndexDistCancelWithPartition(t *testing.T) {
113115
store := realtikvtest.CreateMockStoreAndSetup(t)
114116
if store.Name() != "TiKV" {
115117
t.Skip("TiKV store only")
@@ -150,6 +152,69 @@ func TestAddIndexDistCancel(t *testing.T) {
150152
tk.MustExec(`set global tidb_enable_dist_task=0;`)
151153
}
152154

155+
func TestAddIndexDistCancel(t *testing.T) {
156+
store := realtikvtest.CreateMockStoreAndSetup(t)
157+
tk := testkit.NewTestKit(t, store)
158+
tk.MustExec("drop database if exists addindexlit;")
159+
tk.MustExec("create database addindexlit;")
160+
tk.MustExec("use addindexlit;")
161+
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
162+
tk.MustExec("create table t (a int, b int);")
163+
tk.MustExec("insert into t values (1, 1), (2, 2), (3, 3);")
164+
165+
tk2 := testkit.NewTestKit(t, store)
166+
tk2.MustExec("use addindexlit;")
167+
tk2.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)
168+
tk2.MustExec("create table t2 (a int, b int);")
169+
tk2.MustExec("insert into t2 values (1, 1), (2, 2), (3, 3);")
170+
171+
wg := &sync.WaitGroup{}
172+
wg.Add(2)
173+
go func() {
174+
tk.MustExec("alter table t add index idx(a);")
175+
wg.Done()
176+
}()
177+
go func() {
178+
tk2.MustExec("alter table t2 add index idx_b(b);")
179+
wg.Done()
180+
}()
181+
wg.Wait()
182+
rows := tk.MustQuery("admin show ddl jobs 2;").Rows()
183+
require.Len(t, rows, 2)
184+
require.True(t, strings.Contains(rows[0][12].(string) /* comments */, "ingest"))
185+
require.True(t, strings.Contains(rows[1][12].(string) /* comments */, "ingest"))
186+
require.Equal(t, rows[0][7].(string) /* row_count */, "3")
187+
require.Equal(t, rows[1][7].(string) /* row_count */, "3")
188+
189+
tk.MustExec("set @@global.tidb_enable_dist_task = 1;")
190+
191+
// test cancel is timely
192+
enter := make(chan struct{})
193+
testfailpoint.EnableCall(
194+
t,
195+
"github.com/pingcap/tidb/pkg/lightning/backend/local/beforeExecuteRegionJob",
196+
func(ctx context.Context) {
197+
close(enter)
198+
select {
199+
case <-time.After(time.Second * 30):
200+
case <-ctx.Done():
201+
}
202+
})
203+
wg.Add(1)
204+
go func() {
205+
defer wg.Done()
206+
err := tk2.ExecToErr("alter table t add index idx_ba(b, a);")
207+
require.ErrorContains(t, err, "Cancelled DDL job")
208+
}()
209+
<-enter
210+
jobID := tk.MustQuery("admin show ddl jobs 1;").Rows()[0][0].(string)
211+
now := time.Now()
212+
tk.MustExec("admin cancel ddl jobs " + jobID)
213+
wg.Wait()
214+
// cancel should be timely
215+
require.Less(t, time.Since(now).Seconds(), 20.0)
216+
}
217+
153218
func TestAddIndexDistPauseAndResume(t *testing.T) {
154219
t.Skip("unstable") // TODO(tangenta): fix this unstable test
155220
store := realtikvtest.CreateMockStoreAndSetup(t)

0 commit comments

Comments
 (0)