Skip to content

Commit

Permalink
ddl: avoid commit conflicts when updating/delete from mysql.tidb_ddl_…
Browse files Browse the repository at this point in the history
…reorg. (#38738)

* Added test case

* ddl fix #38669.

The issue was that mysql.tidb_ddl_reorg table was updated by an
inner transaction after the outer transaction started,
which then made a commit conflict in the outer transaction,
when it deleted the same row.

* Fixed typo in comment

* Added test case for #24427

* Disabled tests for CI testing

* Revert "Disabled tests for CI testing"

This reverts commit 17c28f3.

* Revert "Revert "Disabled tests for CI testing""

This reverts commit 65c84d9.

* removed test skips

* Clean up the tidb_ddl_reorg entry after DDL is completed

* Use a cleanup job afterwards instead.

* Fixed test

* Moved cleanup before asyncNotify

* More detailed test failure log

* Refined test error message

* Injecting timoeut to get stack traces from CI

* Updated Debug Dump on timeout

* Delete mulitple entries in tidb_ddl_reorg if needed

* Linting

* Linting

* Added CI debug logs

* Linting + CI debugs

* fixed CI debug

* Try to cleanup also if job.State == synced

* check for non-error of runErr instead of error...

* Use a new session, instead of reusing worker.sess

* Also handle case when job == nil

* Removed CI debug logs

* Misssed change session from w.sess to newly created sess

* Improved TestConcurrentDDLSwitch and added CI debug logs

* Always cleaning up all orphan mysql.tidb_ddl_reorg entries

* linting

* Also cleanup if job is nil

* Updated TestModifyColumnReorgInfo + CI debug logs

* more CI debug

* refactored the cleanupDDLReorgHandle code

* Added missing cleanup in handleDDLJobQueue

* Removed debug panic

* Code cleanup

* Test updates

* Debug cleanup

* Cleaned up test after removal of old non-concurrent DDL code merge

* Linting

* always wrap changes to tidb_ddl_reorg in an own transaction

+ fixed some typos

* Minimum fix

* Always update reorg meta, not only on error

* Issue is here :)

* Fixed newReorgHandler

* Wrapped more tidb_ddl_reorg changes into separate transactions

* linting

* Removed updateDDLReorgStartHandle

* cleanups

* Made runInTxn a method on *session, instead of normal function

* Update test

* Final touches

* Removed duplicate test

* CleanupDDLReorgHandles should only be called from HandleJobDone.

* Variable rename

* Renamed 'delete' variabel name

* Updated test

* small revert

* Removed timeout debugging code

* Simplified the cleanup to only start a new txn and not a new session

* Reverted the change of GetDDLInfo

Co-authored-by: Ti Chi Robot <ti-community-prow-bot@tidb.io>
  • Loading branch information
mjonss and ti-chi-bot authored Jan 11, 2023
1 parent f88714d commit eb35c77
Show file tree
Hide file tree
Showing 13 changed files with 179 additions and 122 deletions.
24 changes: 13 additions & 11 deletions ddl/backfilling.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,7 +146,7 @@ func GetLeaseGoTime(currTime time.Time, lease time.Duration) types.Time {
// Backfilling is time consuming, to accelerate this process, TiDB has built some sub
// workers to do this in the DDL owner node.
//
// DDL owner thread
// DDL owner thread (also see comments before runReorgJob func)
// ^
// | (reorgCtx.doneCh)
// |
Expand Down Expand Up @@ -583,9 +583,10 @@ func (dc *ddlCtx) sendTasksAndWait(scheduler *backfillScheduler, totalAddedCount
err = dc.isReorgRunnable(reorgInfo.Job.ID)
}

// Update the reorg handle that has been processed.
err1 := reorgInfo.UpdateReorgMeta(nextKey, scheduler.sessPool)

if err != nil {
// Update the reorg handle that has been processed.
err1 := reorgInfo.UpdateReorgMeta(nextKey, scheduler.sessPool)
metrics.BatchAddIdxHistogram.WithLabelValues(metrics.LblError).Observe(elapsedTime.Seconds())
logutil.BgLogger().Warn("[ddl] backfill worker handle batch tasks failed",

Expand Down Expand Up @@ -614,7 +615,8 @@ func (dc *ddlCtx) sendTasksAndWait(scheduler *backfillScheduler, totalAddedCount
zap.String("start key", hex.EncodeToString(startKey)),
zap.String("next key", hex.EncodeToString(nextKey)),
zap.Int64("batch added count", taskAddedCount),
zap.String("take time", elapsedTime.String()))
zap.String("take time", elapsedTime.String()),
zap.NamedError("updateHandleError", err1))
return nil
}

Expand Down Expand Up @@ -1320,15 +1322,15 @@ func GetMaxBackfillJob(sess *session, jobID, currEleID int64, currEleKey []byte)
}

// MoveBackfillJobsToHistoryTable moves backfill table jobs to the backfill history table.
func MoveBackfillJobsToHistoryTable(sessCtx sessionctx.Context, bfJob *BackfillJob) error {
sess, ok := sessCtx.(*session)
func MoveBackfillJobsToHistoryTable(sctx sessionctx.Context, bfJob *BackfillJob) error {
s, ok := sctx.(*session)
if !ok {
return errors.Errorf("sess ctx:%#v convert session failed", sessCtx)
return errors.Errorf("sess ctx:%#v convert session failed", sctx)
}

return runInTxn(sess, func(se *session) error {
return s.runInTxn(func(se *session) error {
// TODO: Consider batch by batch update backfill jobs and insert backfill history jobs.
bJobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'",
bJobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s'",
bfJob.JobID, bfJob.EleID, bfJob.EleKey), "update_backfill_job")
if err != nil {
return errors.Trace(err)
Expand All @@ -1342,13 +1344,13 @@ func MoveBackfillJobsToHistoryTable(sessCtx sessionctx.Context, bfJob *BackfillJ
return errors.Trace(err)
}
startTS := txn.StartTS()
err = RemoveBackfillJob(sess, true, bJobs[0])
err = RemoveBackfillJob(se, true, bJobs[0])
if err == nil {
for _, bj := range bJobs {
bj.State = model.JobStateCancelled
bj.FinishTS = startTS
}
err = AddBackfillHistoryJob(sess, bJobs)
err = AddBackfillHistoryJob(se, bJobs)
}
logutil.BgLogger().Info("[ddl] move backfill jobs to history table", zap.Int("job count", len(bJobs)))
return errors.Trace(err)
Expand Down
17 changes: 12 additions & 5 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -806,7 +806,13 @@ func doReorgWorkForModifyColumnMultiSchema(w *worker, d *ddlCtx, t *meta.Meta, j
func doReorgWorkForModifyColumn(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, tbl table.Table,
oldCol, changingCol *model.ColumnInfo, changingIdxs []*model.IndexInfo) (done bool, ver int64, err error) {
job.ReorgMeta.ReorgTp = model.ReorgTypeTxn
rh := newReorgHandler(t, w.sess)
sctx, err1 := w.sessPool.get()
if err1 != nil {
err = errors.Trace(err1)
return
}
defer w.sessPool.put(sctx)
rh := newReorgHandler(newSession(sctx))
dbInfo, err := t.GetDatabase(job.SchemaID)
if err != nil {
return false, ver, errors.Trace(err)
Expand Down Expand Up @@ -1291,8 +1297,8 @@ func (w *updateColumnWorker) getRowRecord(handle kv.Handle, recordKey []byte, ra
if err != nil {
return w.reformatErrors(err)
}
if w.sessCtx.GetSessionVars().StmtCtx.GetWarnings() != nil && len(w.sessCtx.GetSessionVars().StmtCtx.GetWarnings()) != 0 {
warn := w.sessCtx.GetSessionVars().StmtCtx.GetWarnings()
warn := w.sessCtx.GetSessionVars().StmtCtx.GetWarnings()
if len(warn) != 0 {
//nolint:forcetypeassert
recordWarning = errors.Cause(w.reformatErrors(warn[0].Err)).(*terror.Error)
}
Expand Down Expand Up @@ -1376,8 +1382,9 @@ func (w *updateColumnWorker) BackfillDataInTxn(handleRange reorgBackfillTask) (t
taskCtx.nextKey = nextKey
taskCtx.done = taskDone

warningsMap := make(map[errors.ErrorID]*terror.Error, len(rowRecords))
warningsCountMap := make(map[errors.ErrorID]int64, len(rowRecords))
// Optimize for few warnings!
warningsMap := make(map[errors.ErrorID]*terror.Error, 2)
warningsCountMap := make(map[errors.ErrorID]int64, 2)
for _, rowRecord := range rowRecords {
taskCtx.scanCount++

Expand Down
15 changes: 15 additions & 0 deletions ddl/column_type_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2421,3 +2421,18 @@ func TestColumnTypeChangeTimestampToInt(t *testing.T) {
tk.MustExec("alter table t add index idx1(id, c1);")
tk.MustExec("admin check table t")
}

func TestFixDDLTxnWillConflictWithReorgTxn(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

tk.MustExec("create table t (a int)")
tk.MustExec("set global tidb_ddl_enable_fast_reorg = OFF")
tk.MustExec("alter table t add index(a)")
tk.MustExec("set @@sql_mode=''")
tk.MustExec("insert into t values(128),(129)")
tk.MustExec("alter table t modify column a tinyint")

tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1690 2 warnings with this error code, first warning: constant 128 overflows tinyint"))
}
19 changes: 19 additions & 0 deletions ddl/db_partition_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4528,6 +4528,25 @@ func TestPartitionTableWithAnsiQuotes(t *testing.T) {
` PARTITION "pMax" VALUES LESS THAN (MAXVALUE,MAXVALUE))`))
}

func TestAlterModifyPartitionColTruncateWarning(t *testing.T) {
t.Skip("waiting for supporting Modify Partition Column again")
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
schemaName := "truncWarn"
tk.MustExec("create database " + schemaName)
tk.MustExec("use " + schemaName)
tk.MustExec(`set sql_mode = default`)
tk.MustExec(`create table t (a varchar(255)) partition by range columns (a) (partition p1 values less than ("0"), partition p2 values less than ("zzzz"))`)
tk.MustExec(`insert into t values ("123456"),(" 654321")`)
tk.MustContainErrMsg(`alter table t modify a varchar(5)`, "[types:1265]Data truncated for column 'a', value is '")
tk.MustExec(`set sql_mode = ''`)
tk.MustExec(`alter table t modify a varchar(5)`)
// Fix the duplicate warning, see https://github.com/pingcap/tidb/issues/38699
tk.MustQuery(`show warnings`).Check(testkit.Rows(""+
"Warning 1265 Data truncated for column 'a', value is ' 654321'",
"Warning 1265 Data truncated for column 'a', value is ' 654321'"))
}

func TestAlterModifyColumnOnPartitionedTableRename(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
Expand Down
5 changes: 1 addition & 4 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -618,10 +618,7 @@ func TestAddExpressionIndexRollback(t *testing.T) {
// Check whether the reorg information is cleaned up.
err := sessiontxn.NewTxn(context.Background(), ctx)
require.NoError(t, err)
txn, err := ctx.Txn(true)
require.NoError(t, err)
m := meta.NewMeta(txn)
element, start, end, physicalID, err := ddl.NewReorgHandlerForTest(m, testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob)
element, start, end, physicalID, err := ddl.NewReorgHandlerForTest(testkit.NewTestKit(t, store).Session()).GetDDLReorgHandle(currJob)
require.True(t, meta.ErrDDLReorgElementNotExist.Equal(err))
require.Nil(t, element)
require.Nil(t, start)
Expand Down
15 changes: 14 additions & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -1343,7 +1343,7 @@ func GetDDLInfo(s sessionctx.Context) (*Info, error) {
return info, nil
}

_, info.ReorgHandle, _, _, err = newReorgHandler(t, sess).GetDDLReorgHandle(reorgJob)
_, info.ReorgHandle, _, _, err = newReorgHandler(sess).GetDDLReorgHandle(reorgJob)
if err != nil {
if meta.ErrDDLReorgElementNotExist.Equal(err) {
return info, nil
Expand Down Expand Up @@ -1584,6 +1584,19 @@ func (s *session) session() sessionctx.Context {
return s.Context
}

func (s *session) runInTxn(f func(*session) error) (err error) {
err = s.begin()
if err != nil {
return err
}
err = f(s)
if err != nil {
s.rollback()
return
}
return errors.Trace(s.commit())
}

// GetAllHistoryDDLJobs get all the done DDL jobs.
func GetAllHistoryDDLJobs(m *meta.Meta) ([]*model.Job, error) {
iterator, err := GetLastHistoryDDLJobsIterator(m)
Expand Down
1 change: 1 addition & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,6 +699,7 @@ func (w *worker) HandleJobDone(d *ddlCtx, job *model.Job, t *meta.Meta) error {
if err != nil {
return err
}
CleanupDDLReorgHandles(job, w.sess)
asyncNotify(d.ddlJobDoneCh)
return nil
}
Expand Down
30 changes: 15 additions & 15 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -882,7 +882,13 @@ func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Jo
func runReorgJobAndHandleErr(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job,
tbl table.Table, indexInfo *model.IndexInfo, mergingTmpIdx bool) (done bool, ver int64, err error) {
elements := []*meta.Element{{ID: indexInfo.ID, TypeKey: meta.IndexElementKey}}
rh := newReorgHandler(t, w.sess)
sctx, err1 := w.sessPool.get()
if err1 != nil {
err = err1
return
}
defer w.sessPool.put(sctx)
rh := newReorgHandler(newSession(sctx))
dbInfo, err := t.GetDatabase(job.SchemaID)
if err != nil {
return false, ver, errors.Trace(err)
Expand Down Expand Up @@ -1274,13 +1280,10 @@ func (w *baseIndexWorker) String() string {
}

func (w *baseIndexWorker) UpdateTask(bfJob *BackfillJob) error {
sess, ok := w.backfillCtx.sessCtx.(*session)
if !ok {
return errors.Errorf("sess ctx:%#v convert session failed", w.backfillCtx.sessCtx)
}
s := newSession(w.backfillCtx.sessCtx)

return runInTxn(sess, func(se *session) error {
jobs, err := GetBackfillJobs(sess, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d",
return s.runInTxn(func(se *session) error {
jobs, err := GetBackfillJobs(se, BackfillTable, fmt.Sprintf("ddl_job_id = %d and ele_id = %d and ele_key = '%s' and id = %d",
bfJob.JobID, bfJob.EleID, bfJob.EleKey, bfJob.ID), "update_backfill_task")
if err != nil {
return err
Expand All @@ -1297,26 +1300,23 @@ func (w *baseIndexWorker) UpdateTask(bfJob *BackfillJob) error {
return err
}
bfJob.InstanceLease = GetLeaseGoTime(currTime, InstanceLease)
return updateBackfillJob(sess, BackfillTable, bfJob, "update_backfill_task")
return updateBackfillJob(se, BackfillTable, bfJob, "update_backfill_task")
})
}

func (w *baseIndexWorker) FinishTask(bfJob *BackfillJob) error {
sess, ok := w.backfillCtx.sessCtx.(*session)
if !ok {
return errors.Errorf("sess ctx:%#v convert session failed", w.backfillCtx.sessCtx)
}
return runInTxn(sess, func(se *session) error {
s := newSession(w.backfillCtx.sessCtx)
return s.runInTxn(func(se *session) error {
txn, err := se.txn()
if err != nil {
return errors.Trace(err)
}
bfJob.FinishTS = txn.StartTS()
err = RemoveBackfillJob(sess, false, bfJob)
err = RemoveBackfillJob(se, false, bfJob)
if err != nil {
return err
}
return AddBackfillHistoryJob(sess, []*BackfillJob{bfJob})
return AddBackfillHistoryJob(se, []*BackfillJob{bfJob})
})
}

Expand Down
Loading

0 comments on commit eb35c77

Please sign in to comment.