Skip to content

Commit 1b84f38

Browse files
authored
ddl: consider paused job when check runnable (#54419) (#54431)
ref #53246, close #54383
1 parent dc816da commit 1b84f38

File tree

9 files changed

+133
-52
lines changed

9 files changed

+133
-52
lines changed

pkg/ddl/callback.go

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -60,9 +60,9 @@ type Callback interface {
6060
// OnWatched is called after watching owner is completed.
6161
OnWatched(ctx context.Context)
6262
// OnGetJobBefore is called before getting job.
63-
OnGetJobBefore(jobType string)
63+
OnGetJobBefore()
6464
// OnGetJobAfter is called after getting job.
65-
OnGetJobAfter(jobType string, job *model.Job)
65+
OnGetJobAfter(job *model.Job)
6666
}
6767

6868
// BaseCallback implements Callback.OnChanged interface.
@@ -100,12 +100,12 @@ func (*BaseCallback) OnWatched(_ context.Context) {
100100
}
101101

102102
// OnGetJobBefore implements Callback.OnGetJobBefore interface.
103-
func (*BaseCallback) OnGetJobBefore(_ string) {
103+
func (*BaseCallback) OnGetJobBefore() {
104104
// Nothing to do.
105105
}
106106

107107
// OnGetJobAfter implements Callback.OnGetJobAfter interface.
108-
func (*BaseCallback) OnGetJobAfter(_ string, _ *model.Job) {
108+
func (*BaseCallback) OnGetJobAfter(_ *model.Job) {
109109
// Nothing to do.
110110
}
111111

pkg/ddl/column.go

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -753,6 +753,7 @@ func (w *worker) doModifyColumnTypeWithData(
753753
return ver, errors.Trace(err)
754754
}
755755
job.SchemaState = model.StateWriteOnly
756+
failpoint.InjectCall("afterModifyColumnStateDeleteOnly", job.ID)
756757
case model.StateWriteOnly:
757758
// write only -> reorganization
758759
updateChangingObjState(changingCol, changingIdxs, model.StateWriteReorganization)

pkg/ddl/ddl_running_jobs.go

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -233,11 +233,23 @@ func (j *runningJobs) addRunning(jobID int64, involves []model.InvolvingSchemaIn
233233
}
234234
}
235235

236+
func (j *runningJobs) finishOrPendJob(jobID int64, involves []model.InvolvingSchemaInfo, moveToPending bool) {
237+
j.mu.Lock()
238+
defer j.mu.Unlock()
239+
j.removeRunningWithoutLock(jobID, involves)
240+
if moveToPending {
241+
j.addPendingWithoutLock(involves)
242+
}
243+
}
244+
236245
// removeRunning can be concurrently called with add and checkRunnable.
237246
func (j *runningJobs) removeRunning(jobID int64, involves []model.InvolvingSchemaInfo) {
238247
j.mu.Lock()
239248
defer j.mu.Unlock()
249+
j.removeRunningWithoutLock(jobID, involves)
250+
}
240251

252+
func (j *runningJobs) removeRunningWithoutLock(jobID int64, involves []model.InvolvingSchemaInfo) {
241253
if intest.InTest {
242254
if _, ok := j.ids[jobID]; !ok {
243255
panic(fmt.Sprintf("job %d is not running", jobID))
@@ -296,6 +308,10 @@ func (j *runningJobs) addPending(involves []model.InvolvingSchemaInfo) {
296308
j.mu.Lock()
297309
defer j.mu.Unlock()
298310

311+
j.addPendingWithoutLock(involves)
312+
}
313+
314+
func (j *runningJobs) addPendingWithoutLock(involves []model.InvolvingSchemaInfo) {
299315
for _, info := range involves {
300316
if info.Database != model.InvolvingNone {
301317
if _, ok := j.pending.schemas[info.Database]; !ok {

pkg/ddl/ddl_worker_test.go

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -135,7 +135,7 @@ func TestParallelDDL(t *testing.T) {
135135
}
136136

137137
once1 := sync.Once{}
138-
tc.OnGetJobBeforeExported = func(string) {
138+
tc.OnGetJobBeforeExported = func() {
139139
once1.Do(func() {
140140
for {
141141
tk := testkit.NewTestKit(t, store)

pkg/ddl/job_table.go

Lines changed: 40 additions & 35 deletions
Original file line numberDiff line numberDiff line change
@@ -185,51 +185,47 @@ func (s *jobScheduler) close() {
185185
}
186186

187187
// getJob reads tidb_ddl_job and returns the first runnable DDL job.
188-
func (s *jobScheduler) getJob(se *sess.Session, tp jobType) (*model.Job, error) {
188+
func (s *jobScheduler) getJob(se *sess.Session) (*model.Job, bool, error) {
189189
defer s.runningJobs.resetAllPending()
190190

191-
not := "not"
192-
label := "get_job_general"
193-
if tp == jobTypeReorg {
194-
not = ""
195-
label = "get_job_reorg"
196-
}
197-
const getJobSQL = `select job_meta, processing from mysql.tidb_ddl_job where job_id in
191+
const getJobSQL = `select job_meta, processing, reorg from mysql.tidb_ddl_job where job_id in
198192
(select min(job_id) from mysql.tidb_ddl_job group by schema_ids, table_ids, processing)
199-
and %s reorg %s order by processing desc, job_id`
193+
%s order by processing desc, job_id`
200194
var excludedJobIDs string
201195
if ids := s.runningJobs.allIDs(); len(ids) > 0 {
202196
excludedJobIDs = fmt.Sprintf("and job_id not in (%s)", ids)
203197
}
204-
sql := fmt.Sprintf(getJobSQL, not, excludedJobIDs)
205-
rows, err := se.Execute(context.Background(), sql, label)
198+
sql := fmt.Sprintf(getJobSQL, excludedJobIDs)
199+
rows, err := se.Execute(context.Background(), sql, "get_job")
206200
if err != nil {
207-
return nil, errors.Trace(err)
201+
return nil, false, errors.Trace(err)
208202
}
209203
for _, row := range rows {
210204
jobBinary := row.GetBytes(0)
211205
isJobProcessing := row.GetInt64(1) == 1
206+
isReorg := row.GetInt64(2) != 0
212207

213208
job := model.Job{}
214209
err = job.Decode(jobBinary)
215210
if err != nil {
216-
return nil, errors.Trace(err)
211+
return nil, isReorg, errors.Trace(err)
217212
}
218213

214+
involving := job.GetInvolvingSchemaInfo()
219215
isRunnable, err := s.processJobDuringUpgrade(se, &job)
220216
if err != nil {
221-
return nil, errors.Trace(err)
217+
return nil, isReorg, errors.Trace(err)
222218
}
223219
if !isRunnable {
220+
s.runningJobs.addPending(involving)
224221
continue
225222
}
226223

227224
// The job has already been picked up, just return to continue it.
228225
if isJobProcessing {
229-
return &job, nil
226+
return &job, isReorg, nil
230227
}
231228

232-
involving := job.GetInvolvingSchemaInfo()
233229
if !s.runningJobs.checkRunnable(job.ID, involving) {
234230
s.runningJobs.addPending(involving)
235231
continue
@@ -241,11 +237,11 @@ func (s *jobScheduler) getJob(se *sess.Session, tp jobType) (*model.Job, error)
241237
zap.Error(err),
242238
zap.Stringer("job", &job))
243239
s.runningJobs.addPending(involving)
244-
return nil, errors.Trace(err)
240+
return nil, isReorg, errors.Trace(err)
245241
}
246-
return &job, nil
242+
return &job, isReorg, nil
247243
}
248-
return nil, nil
244+
return nil, false, nil
249245
}
250246

251247
func hasSysDB(job *model.Job) bool {
@@ -394,8 +390,7 @@ func (s *jobScheduler) startDispatch() error {
394390
continue
395391
}
396392
failpoint.InjectCall("beforeAllLoadDDLJobAndRun")
397-
s.loadDDLJobAndRun(se, s.generalDDLWorkerPool, jobTypeGeneral)
398-
s.loadDDLJobAndRun(se, s.reorgWorkerPool, jobTypeReorg)
393+
s.loadDDLJobAndRun(se)
399394
}
400395
}
401396

@@ -436,30 +431,32 @@ func (s *jobScheduler) checkAndUpdateClusterState(needUpdate bool) error {
436431
return nil
437432
}
438433

439-
func (s *jobScheduler) loadDDLJobAndRun(se *sess.Session, pool *workerPool, tp jobType) {
440-
wk, err := pool.get()
441-
if err != nil || wk == nil {
442-
logutil.DDLLogger().Debug(fmt.Sprintf("[ddl] no %v worker available now", pool.tp()), zap.Error(err))
443-
return
444-
}
445-
434+
func (s *jobScheduler) loadDDLJobAndRun(se *sess.Session) {
446435
s.mu.RLock()
447-
s.mu.hook.OnGetJobBefore(pool.tp().String())
436+
s.mu.hook.OnGetJobBefore()
448437
s.mu.RUnlock()
449438

450439
startTime := time.Now()
451-
job, err := s.getJob(se, tp)
440+
job, isReorg, err := s.getJob(se)
452441
if job == nil || err != nil {
453442
if err != nil {
454-
wk.jobLogger(job).Warn("get job met error", zap.Duration("take time", time.Since(startTime)), zap.Error(err))
443+
logutil.DDLLogger().Warn("get job met error", zap.Duration("take time", time.Since(startTime)), zap.Error(err))
455444
}
456-
pool.put(wk)
457445
return
458446
}
459447
s.mu.RLock()
460-
s.mu.hook.OnGetJobAfter(pool.tp().String(), job)
448+
s.mu.hook.OnGetJobAfter(job)
461449
s.mu.RUnlock()
462450

451+
pool := s.generalDDLWorkerPool
452+
if isReorg {
453+
pool = s.reorgWorkerPool
454+
}
455+
wk, err := pool.get()
456+
if err != nil || wk == nil {
457+
logutil.DDLLogger().Debug(fmt.Sprintf("[ddl] no %v worker available now", pool.tp()), zap.Error(err))
458+
return
459+
}
463460
s.delivery2Worker(wk, pool, job)
464461
}
465462

@@ -526,10 +523,18 @@ func (s *jobScheduler) delivery2Worker(wk *worker, pool *workerPool, job *model.
526523
jobID, involvedSchemaInfos := job.ID, job.GetInvolvingSchemaInfo()
527524
s.runningJobs.addRunning(jobID, involvedSchemaInfos)
528525
metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Inc()
529-
s.wg.RunWithLog(func() {
526+
s.wg.Run(func() {
530527
defer func() {
528+
r := recover()
529+
if r != nil {
530+
logutil.DDLLogger().Error("panic in delivery2Worker", zap.Any("recover", r), zap.Stack("stack"))
531+
}
531532
failpoint.InjectCall("afterDelivery2Worker", job)
532-
s.runningJobs.removeRunning(jobID, involvedSchemaInfos)
533+
// Because there is a gap between `allIDs()` and `checkRunnable()`,
534+
// we append unfinished job to pending atomically to prevent `getJob()`
535+
// chosing another runnable job that involves the same schema object.
536+
moveRunningJobsToPending := r != nil || (job != nil && !job.IsFinished())
537+
s.runningJobs.finishOrPendJob(jobID, involvedSchemaInfos, moveRunningJobsToPending)
533538
asyncNotify(s.ddlJobNotifyCh)
534539
metrics.DDLRunningJobCount.WithLabelValues(pool.tp().String()).Dec()
535540
pool.put(wk)

pkg/ddl/job_table_test.go

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -65,7 +65,7 @@ func TestDDLScheduling(t *testing.T) {
6565
var wg util.WaitGroupWrapper
6666
wg.Add(1)
6767
var once sync.Once
68-
hook.OnGetJobBeforeExported = func(jobType string) {
68+
hook.OnGetJobBeforeExported = func() {
6969
once.Do(func() {
7070
for i, job := range ddlJobs {
7171
wg.Run(func() {
@@ -91,7 +91,7 @@ func TestDDLScheduling(t *testing.T) {
9191
}
9292

9393
record := make([]int64, 0, 16)
94-
hook.OnGetJobAfterExported = func(jobType string, job *model.Job) {
94+
hook.OnGetJobAfterExported = func(job *model.Job) {
9595
// record the job schedule order
9696
record = append(record, job.ID)
9797
}

pkg/ddl/tests/adminpause/BUILD.bazel

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -29,7 +29,7 @@ go_test(
2929
],
3030
embed = [":adminpause"],
3131
flaky = True,
32-
shard_count = 14,
32+
shard_count = 15,
3333
deps = [
3434
"//pkg/config",
3535
"//pkg/ddl",

pkg/ddl/tests/adminpause/pause_resume_test.go

Lines changed: 60 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,9 @@ import (
2020
"strconv"
2121
"sync"
2222
"testing"
23+
"time"
2324

25+
"github.com/pingcap/failpoint"
2426
testddlutil "github.com/pingcap/tidb/pkg/ddl/testutil"
2527
"github.com/pingcap/tidb/pkg/ddl/util/callback"
2628
"github.com/pingcap/tidb/pkg/domain"
@@ -123,7 +125,7 @@ func pauseResumeAndCancel(t *testing.T, stmtKit *testkit.TestKit, adminCommandKi
123125
var isCancelled = false
124126
var cancelResult []sqlexec.RecordSet
125127
var cancelErr error
126-
var cancelFunc = func(jobType string) {
128+
var cancelFunc = func() {
127129
adminCommandMutex.Lock()
128130
defer adminCommandMutex.Unlock()
129131
if isPaused && isResumed && !isCancelled {
@@ -354,3 +356,60 @@ func TestPauseResumeCancelAndRerunPartitionTableStmt(t *testing.T) {
354356

355357
Logger.Info("TestPauseResumeCancelAndRerunPartitionTableStmt: all cases finished.")
356358
}
359+
360+
func TestPauseJobDependency(t *testing.T) {
361+
store := testkit.CreateMockStore(t)
362+
tk := testkit.NewTestKit(t, store)
363+
tk.MustExec("use test")
364+
tk2 := testkit.NewTestKit(t, store)
365+
tk2.MustExec("use test")
366+
367+
tk.MustExec("create table t (a int, b int);")
368+
tk.MustExec("insert into t values (1, 1);")
369+
370+
afterPause := make(chan struct{})
371+
afterAddCol := make(chan struct{})
372+
startAddCol := make(chan struct{})
373+
var (
374+
modifyJobID int64
375+
errModCol error
376+
errAddCol error
377+
)
378+
once := sync.Once{}
379+
failpoint.EnableCall("github.com/pingcap/tidb/pkg/ddl/afterModifyColumnStateDeleteOnly", func(jobID int64) {
380+
once.Do(func() {
381+
modifyJobID = jobID
382+
tk2.MustExec(fmt.Sprintf("admin pause ddl jobs %d", jobID))
383+
afterPause <- struct{}{}
384+
})
385+
})
386+
wg := sync.WaitGroup{}
387+
wg.Add(2)
388+
go func() {
389+
defer wg.Done()
390+
// Will stuck because the job is paused.
391+
errModCol = tk.ExecToErr("alter table t modify column b tinyint;")
392+
}()
393+
go func() {
394+
defer wg.Done()
395+
<-afterPause
396+
// This should be blocked because they handle the same table.
397+
startAddCol <- struct{}{}
398+
errAddCol = tk2.ExecToErr("alter table t add column c int;")
399+
afterAddCol <- struct{}{}
400+
}()
401+
<-startAddCol
402+
select {
403+
case <-afterAddCol:
404+
t.Logf("add column DDL on same table should be blocked")
405+
t.FailNow()
406+
case <-time.After(3 * time.Second):
407+
tk3 := testkit.NewTestKit(t, store)
408+
tk3.MustExec("use test")
409+
tk3.MustExec(fmt.Sprintf("admin resume ddl jobs %d", modifyJobID))
410+
<-afterAddCol
411+
}
412+
wg.Wait()
413+
require.NoError(t, errModCol)
414+
require.NoError(t, errAddCol)
415+
}

pkg/ddl/util/callback/callback.go

Lines changed: 8 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -55,8 +55,8 @@ type TestDDLCallback struct {
5555
onJobUpdated func(*model.Job)
5656
OnJobUpdatedExported atomic.Pointer[func(*model.Job)]
5757
onWatched func(ctx context.Context)
58-
OnGetJobBeforeExported func(string)
59-
OnGetJobAfterExported func(string, *model.Job)
58+
OnGetJobBeforeExported func()
59+
OnGetJobAfterExported func(*model.Job)
6060
OnJobSchemaStateChanged func(int64)
6161

6262
OnUpdateReorgInfoExported func(job *model.Job, pid int64)
@@ -146,21 +146,21 @@ func (tc *TestDDLCallback) OnWatched(ctx context.Context) {
146146
}
147147

148148
// OnGetJobBefore implements Callback.OnGetJobBefore interface.
149-
func (tc *TestDDLCallback) OnGetJobBefore(jobType string) {
149+
func (tc *TestDDLCallback) OnGetJobBefore() {
150150
if tc.OnGetJobBeforeExported != nil {
151-
tc.OnGetJobBeforeExported(jobType)
151+
tc.OnGetJobBeforeExported()
152152
return
153153
}
154-
tc.BaseCallback.OnGetJobBefore(jobType)
154+
tc.BaseCallback.OnGetJobBefore()
155155
}
156156

157157
// OnGetJobAfter implements Callback.OnGetJobAfter interface.
158-
func (tc *TestDDLCallback) OnGetJobAfter(jobType string, job *model.Job) {
158+
func (tc *TestDDLCallback) OnGetJobAfter(job *model.Job) {
159159
if tc.OnGetJobAfterExported != nil {
160-
tc.OnGetJobAfterExported(jobType, job)
160+
tc.OnGetJobAfterExported(job)
161161
return
162162
}
163-
tc.BaseCallback.OnGetJobAfter(jobType, job)
163+
tc.BaseCallback.OnGetJobAfter(job)
164164
}
165165

166166
// Clone copies the callback and take its reference

0 commit comments

Comments
 (0)