-
Notifications
You must be signed in to change notification settings - Fork 6k
ddl: update row count periodically when running reorg job #60828
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
14b6fd7
a66875b
5fa7cc4
b483955
e12cc4f
6de8726
7fbfc7e
d328b93
0604eb0
0ca1d39
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -96,8 +96,13 @@ type jobContext struct { | |
logger *zap.Logger | ||
|
||
// per job step fields, they will be changed on each call of transitOneJobStep. | ||
// stepCtx is initilaized and destroyed for each job step except reorg job, | ||
// which returns timeout error periodically. | ||
stepCtx context.Context | ||
stepCtxCancel context.CancelCauseFunc | ||
reorgTimeoutOccurred bool | ||
inInnerRunOneJobStep bool // Only used for multi-schema change DDL job. | ||
|
||
stepCtx context.Context | ||
metaMut *meta.Mutator | ||
// decoded JobArgs, we store it here to avoid decoding it multiple times and | ||
// pass some runtime info specific to some job type. | ||
|
@@ -108,6 +113,32 @@ type jobContext struct { | |
lockStartTime time.Time | ||
} | ||
|
||
func (c *jobContext) shouldPollDDLJob() bool { | ||
// If we are in multi-schema change DDL and this is not the outermost | ||
// runOneJobStep, we should not start a goroutine to poll the ddl job. | ||
return !c.inInnerRunOneJobStep | ||
} | ||
|
||
func (c *jobContext) initStepCtx() { | ||
if c.stepCtx == nil { | ||
stepCtx, cancel := context.WithCancelCause(c.ctx) | ||
c.stepCtx = stepCtx | ||
c.stepCtxCancel = cancel | ||
} | ||
} | ||
|
||
func (c *jobContext) cleanStepCtx() { | ||
// reorgTimeoutOccurred indicates whether the current reorg process | ||
// was temporarily exit due to a timeout condition. When set to true, | ||
// it prevents premature cleanup of step context. | ||
if c.reorgTimeoutOccurred { | ||
c.reorgTimeoutOccurred = false // reset flag | ||
return | ||
} | ||
c.stepCtxCancel(context.Canceled) | ||
c.stepCtx = nil // unset stepCtx for the next step initialization | ||
} | ||
|
||
func (c *jobContext) getAutoIDRequirement() autoid.Requirement { | ||
return &asAutoIDRequirement{ | ||
store: c.store, | ||
|
@@ -584,7 +615,7 @@ func (w *worker) transitOneJobStep( | |
}() | ||
// If running job meets error, we will save this error in job Error and retry | ||
// later if the job is not cancelled. | ||
schemaVer, updateRawArgs, runJobErr := w.runOneJobStep(jobCtx, job, jobCtx.sysTblMgr) | ||
schemaVer, updateRawArgs, runJobErr := w.runOneJobStep(jobCtx, job) | ||
|
||
failpoint.InjectCall("afterRunOneJobStep", job) | ||
|
||
|
@@ -774,7 +805,6 @@ func (*worker) processJobPausingRequest(jobCtx *jobContext, job *model.Job) (isR | |
func (w *worker) runOneJobStep( | ||
jobCtx *jobContext, | ||
job *model.Job, | ||
sysTblMgr systable.Manager, | ||
) (ver int64, updateRawArgs bool, err error) { | ||
defer tidbutil.Recover(metrics.LabelDDLWorker, fmt.Sprintf("%s runOneJobStep", w), | ||
func() { | ||
|
@@ -784,8 +814,10 @@ func (w *worker) runOneJobStep( | |
// Mock for run ddl job panic. | ||
failpoint.Inject("mockPanicInRunDDLJob", func(failpoint.Value) {}) | ||
|
||
failpoint.InjectCall("onRunOneJobStep") | ||
if job.Type != model.ActionMultiSchemaChange { | ||
jobCtx.logger.Info("run one job step", zap.String("job", job.String())) | ||
failpoint.InjectCall("onRunOneJobStep") | ||
} | ||
timeStart := time.Now() | ||
if job.RealStartTS == 0 { | ||
|
@@ -809,27 +841,20 @@ func (w *worker) runOneJobStep( | |
return ver, false, err | ||
} | ||
|
||
// when sysTblMgr is nil, clean up the job step context just for clearness. | ||
// Otherwise, we are in multi-schema change DDL and this is not the outermost | ||
// runOneJobStep, we should keep the job step context. | ||
if sysTblMgr != nil { | ||
jobCtx.stepCtx = nil | ||
} | ||
|
||
// It would be better to do the positive check, but no idea to list all valid states here now. | ||
if job.IsRollingback() { | ||
// when rolling back, we use worker context to process. | ||
jobCtx.stepCtx = w.workCtx | ||
} else { | ||
job.State = model.JobStateRunning | ||
|
||
if sysTblMgr != nil { | ||
if jobCtx.shouldPollDDLJob() { | ||
failpoint.InjectCall("beforePollDDLJob") | ||
stopCheckingJobCancelled := make(chan struct{}) | ||
defer close(stopCheckingJobCancelled) | ||
|
||
var cancelStep context.CancelCauseFunc | ||
jobCtx.stepCtx, cancelStep = context.WithCancelCause(jobCtx.ctx) | ||
defer cancelStep(context.Canceled) | ||
jobCtx.initStepCtx() | ||
defer jobCtx.cleanStepCtx() | ||
w.wg.Run(func() { | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. every time we enter runOneJobStep, we will refresh the job, so if we change back to the old bahavior, there is no need to keep this routine but if we do remove this routine, will cancel be blocked? as fixed in #56404 cc @lance6716 There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
In this PR, I see in line 886 still calls |
||
ticker := time.NewTicker(2 * time.Second) | ||
defer ticker.Stop() | ||
|
@@ -840,7 +865,7 @@ func (w *worker) runOneJobStep( | |
return | ||
case <-ticker.C: | ||
failpoint.InjectCall("checkJobCancelled", job) | ||
latestJob, err := sysTblMgr.GetJobByID(w.workCtx, job.ID) | ||
latestJob, err := jobCtx.sysTblMgr.GetJobByID(w.workCtx, job.ID) | ||
if goerrors.Is(err, systable.ErrNotFound) { | ||
logutil.DDLLogger().Info( | ||
"job not found, might already finished", | ||
|
@@ -858,13 +883,13 @@ func (w *worker) runOneJobStep( | |
logutil.DDLLogger().Info("job is cancelled", | ||
zap.Int64("job_id", job.ID), | ||
zap.Stringer("state", latestJob.State)) | ||
cancelStep(dbterror.ErrCancelledDDLJob) | ||
jobCtx.stepCtxCancel(dbterror.ErrCancelledDDLJob) | ||
return | ||
case model.JobStatePausing, model.JobStatePaused: | ||
logutil.DDLLogger().Info("job is paused", | ||
zap.Int64("job_id", job.ID), | ||
zap.Stringer("state", latestJob.State)) | ||
cancelStep(dbterror.ErrPausedDDLJob.FastGenByArgs(job.ID)) | ||
jobCtx.stepCtxCancel(dbterror.ErrPausedDDLJob.FastGenByArgs(job.ID)) | ||
return | ||
case model.JobStateDone, model.JobStateSynced: | ||
return | ||
|
@@ -1082,6 +1107,11 @@ func updateGlobalVersionAndWaitSynced( | |
var err error | ||
|
||
if latestSchemaVersion == 0 { | ||
// If the DDL step is still in progress (e.g., during reorg timeout), | ||
// skip logging to avoid generating redundant entries. | ||
if jobCtx.stepCtx != nil { | ||
return nil | ||
} | ||
logutil.DDLLogger().Info("schema version doesn't change", zap.Int64("jobID", job.ID)) | ||
return nil | ||
} | ||
|
Uh oh!
There was an error while loading. Please reload this page.