From ce74359d5b14dbbb6eb437c7809cbf2a5ddb9742 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 29 Nov 2018 01:48:28 +0800 Subject: [PATCH 01/10] fix cancel drop index error --- ddl/db_test.go | 86 +++++++++++++++++++++++++++++++++++++++++++++ ddl/index.go | 5 +++ ddl/rollingback.go | 47 +++++++++++++++++++++++++ util/admin/admin.go | 11 ++++++ 4 files changed, 149 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index 7db262f8852b9..dccd8112da3c4 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -518,6 +518,92 @@ func (s *testDBSuite) TestCancelAddIndex1(c *C) { s.mustExec(c, "alter table t drop index idx_c2") } +// TestCancelDropIndex tests cancel ddl job which type is drop index. +func (s *testDBSuite) TestCancelDropIndex(c *C) { + s.tk = testkit.NewTestKit(c, s.store) + s.mustExec(c, "use test_db") + s.mustExec(c, "drop table if exists t") + s.mustExec(c, "create table t(c1 int, c2 int)") + defer s.mustExec(c, "drop table t;") + for i := 0; i < 5; i++ { + s.mustExec(c, "insert into t values (?, ?)", i, i) + } + + testCases := []struct { + needAddIndex bool + jobState model.JobState + JobSchemaState model.SchemaState + cancelSucc bool + }{ + // model.JobStateNone means the jobs is canceled before the first run. + {true, model.JobStateNone, model.StateNone, true}, + {false, model.JobStateRunning, model.StateWriteOnly, true}, + {false, model.JobStateRunning, model.StateDeleteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteReorganization, false}, + } + + var checkErr error + oldReorgWaitTimeout := ddl.ReorgWaitTimeout + ddl.ReorgWaitTimeout = 50 * time.Millisecond + defer func() { ddl.ReorgWaitTimeout = oldReorgWaitTimeout }() + hook := &ddl.TestDDLCallback{} + var jobID int64 + for _, testCase := range testCases { + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.Type == model.ActionDropIndex && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { + jobID = job.ID + jobIDs := []int64{job.ID} + hookCtx := mock.NewContext() + hookCtx.Store = s.store + err := hookCtx.NewTxn() + if err != nil { + checkErr = errors.Trace(err) + return + } + errs, err := admin.CancelJobs(hookCtx.Txn(true), jobIDs) + if err != nil { + checkErr = errors.Trace(err) + return + } + + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + + checkErr = hookCtx.Txn(true).Commit(context.Background()) + } + } + s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + if testCase.needAddIndex { + s.mustExec(c, "alter table t add index idx_c2(c2)") + } + rs, err := s.tk.Exec("alter table t drop index idx_c2") + if rs != nil { + rs.Close() + } + + t := s.testGetTable(c, "t") + indexInfo := ddl.FindIndexByName("idx_c2", t.Meta().Indices) + if testCase.cancelSucc { + c.Assert(checkErr, IsNil) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:12]cancelled DDL job") + + c.Assert(indexInfo, NotNil) + c.Assert(indexInfo.State, Equals, model.StatePublic) + } else { + err1 := admin.ErrCannotCancelDDLJob.GenWithStackByArgs(jobID) + c.Assert(err, IsNil) + c.Assert(checkErr, NotNil) + c.Assert(checkErr.Error(), Equals, err1.Error()) + + c.Assert(indexInfo, IsNil) + } + } + s.dom.DDL().(ddl.DDLForTest).SetHook(&ddl.TestDDLCallback{}) +} + func (s *testDBSuite) TestAddAnonymousIndex(c *C) { s.tk = testkit.NewTestKit(c, s.store) s.tk.MustExec("use " + s.schemaName) diff --git a/ddl/index.go b/ddl/index.go index acbb737ffd584..cc081fe288078 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -1179,6 +1179,11 @@ func findIndexByName(idxName string, indices []*model.IndexInfo) *model.IndexInf return nil } +// FindIndexByName exports for testing. +func FindIndexByName(idxName string, indices []*model.IndexInfo) *model.IndexInfo { + return findIndexByName(idxName, indices) +} + func allocateIndexID(tblInfo *model.TableInfo) int64 { tblInfo.MaxIndexID++ return tblInfo.MaxIndexID diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 74f9513a6b0c9..d286a3cd2f3ef 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -117,6 +117,51 @@ func rollingbackAddColumn(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, errCancelledDDLJob } +func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { + var indexName model.CIStr + err = job.DecodeArgs(&indexName) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + schemaID := job.SchemaID + tblInfo, err := getTableInfo(t, job, schemaID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + indexInfo := findIndexByName(indexName.L, tblInfo.Indices) + if indexInfo == nil { + job.State = model.JobStateCancelled + return ver, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + } + + originalState := indexInfo.State + switch indexInfo.State { + case model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: + // we can not rollback now, so just continue to drop index. + job.State = model.JobStateRunning + case model.StatePublic, model.StateWriteOnly: + job.State = model.JobStateRollbackDone + indexInfo.State = model.StatePublic + default: + return ver, ErrInvalidIndexState.GenWithStack("invalid index state %v", indexInfo.State) + } + + job.SchemaState = indexInfo.State + job.Args = []interface{}{indexInfo.Name} + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + if job.State == model.JobStateRollbackDone { + job.FinishTableJob(model.JobStateRollbackDone, model.StatePublic, ver, tblInfo) + } + return ver, errCancelledDDLJob +} + func rollingbackAddindex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, err error) { // If the value of SnapshotVer isn't zero, it means the work is backfilling the indexes. if job.SchemaState == model.StateWriteReorganization && job.SnapshotVer != 0 { @@ -137,6 +182,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) ver, err = rollingbackAddColumn(t, job) case model.ActionAddIndex: ver, err = rollingbackAddindex(w, d, t, job) + case model.ActionDropIndex: + ver, err = rollingbackDropIndex(t, job) default: job.State = model.JobStateCancelled err = errCancelledDDLJob diff --git a/util/admin/admin.go b/util/admin/admin.go index 1c4fa7107a0fd..3e5fa0e8c56c4 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -113,6 +113,14 @@ func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { if job.IsCancelled() || job.IsRollingback() || job.IsRollbackDone() { continue } + switch job.Type { + case model.ActionDropIndex: + if job.SchemaState == model.StateDeleteOnly || + job.SchemaState == model.StateDeleteReorganization { + errs[i] = ErrCannotCancelDDLJob.GenWithStackByArgs(id) + continue + } + } job.State = model.JobStateCancelling // Make sure RawArgs isn't overwritten. err := job.DecodeArgs(job.RawArgs) @@ -701,6 +709,7 @@ const ( codeInvalidColumnState = 3 codeDDLJobNotFound = 4 codeCancelFinishedJob = 5 + codeCannotCancelDDLJob = 6 ) var ( @@ -712,4 +721,6 @@ var ( ErrDDLJobNotFound = terror.ClassAdmin.New(codeDDLJobNotFound, "DDL Job:%v not found") // ErrCancelFinishedDDLJob returns when cancel a finished ddl job. ErrCancelFinishedDDLJob = terror.ClassAdmin.New(codeCancelFinishedJob, "This job:%v is finished, so can't be cancelled") + // ErrCannotCancelDDLJob returns when cancel a almost finished ddl job, because cancel in now may cause data inconsistency. + ErrCannotCancelDDLJob = terror.ClassAdmin.New(codeCannotCancelDDLJob, "This job:%v is almost finished, can't be cancelled now") ) From d8f4f56de52ea23658526239131a21e5c8043373 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 29 Nov 2018 11:13:19 +0800 Subject: [PATCH 02/10] add test --- ddl/db_test.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index dccd8112da3c4..fd8315e533f75 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -602,6 +602,8 @@ func (s *testDBSuite) TestCancelDropIndex(c *C) { } } s.dom.DDL().(ddl.DDLForTest).SetHook(&ddl.TestDDLCallback{}) + s.mustExec(c, "alter table t add index idx_c2(c2)") + s.mustExec(c, "alter table t drop index idx_c2") } func (s *testDBSuite) TestAddAnonymousIndex(c *C) { From ce8171991e9d25eb3e3dffb4eb37b4d136344616 Mon Sep 17 00:00:00 2001 From: crazycs Date: Thu, 29 Nov 2018 14:43:54 +0800 Subject: [PATCH 03/10] remove test --- ddl/ddl_worker_test.go | 39 ++++++++++----------------------------- 1 file changed, 10 insertions(+), 29 deletions(-) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 2c931906549d5..b76a2aeed87a2 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -331,7 +331,6 @@ type testCancelJob struct { func buildCancelJobTests(firstID int64) []testCancelJob { err := errCancelledDDLJob - errs := []error{err} noErrs := []error{nil} tests := []testCancelJob{ {act: model.ActionAddIndex, jobIDs: []int64{firstID + 1}, cancelRetErrs: noErrs, cancelState: model.StateDeleteOnly, ddlRetErr: err}, @@ -339,18 +338,15 @@ func buildCancelJobTests(firstID int64) []testCancelJob { {act: model.ActionAddIndex, jobIDs: []int64{firstID + 3}, cancelRetErrs: noErrs, cancelState: model.StateWriteReorganization, ddlRetErr: err}, {act: model.ActionAddIndex, jobIDs: []int64{firstID + 4}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 4)}, cancelState: model.StatePublic, ddlRetErr: err}, - // TODO: after fix drop index and create table rollback bug, the below test cases maybe need to change. - {act: model.ActionDropIndex, jobIDs: []int64{firstID + 5}, cancelRetErrs: errs, cancelState: model.StateWriteOnly, ddlRetErr: err}, - {act: model.ActionDropIndex, jobIDs: []int64{firstID + 6}, cancelRetErrs: errs, cancelState: model.StateDeleteOnly, ddlRetErr: err}, - {act: model.ActionDropIndex, jobIDs: []int64{firstID + 7}, cancelRetErrs: errs, cancelState: model.StateDeleteReorganization, ddlRetErr: err}, - {act: model.ActionDropIndex, jobIDs: []int64{firstID + 8}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 8)}, cancelState: model.StateNone, ddlRetErr: err}, + // Test cancel drop index job , see TestCancelDropIndex. + // TODO: add create table back after we fix the cancel bug. //{act: model.ActionCreateTable, jobIDs: []int64{firstID + 9}, cancelRetErrs: noErrs, cancelState: model.StatePublic, ddlRetErr: err}, - {act: model.ActionAddColumn, jobIDs: []int64{firstID + 9}, cancelRetErrs: noErrs, cancelState: model.StateDeleteOnly, ddlRetErr: err}, - {act: model.ActionAddColumn, jobIDs: []int64{firstID + 10}, cancelRetErrs: noErrs, cancelState: model.StateWriteOnly, ddlRetErr: err}, - {act: model.ActionAddColumn, jobIDs: []int64{firstID + 11}, cancelRetErrs: noErrs, cancelState: model.StateWriteReorganization, ddlRetErr: err}, - {act: model.ActionAddColumn, jobIDs: []int64{firstID + 12}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 12)}, cancelState: model.StatePublic, ddlRetErr: err}, + {act: model.ActionAddColumn, jobIDs: []int64{firstID + 5}, cancelRetErrs: noErrs, cancelState: model.StateDeleteOnly, ddlRetErr: err}, + {act: model.ActionAddColumn, jobIDs: []int64{firstID + 6}, cancelRetErrs: noErrs, cancelState: model.StateWriteOnly, ddlRetErr: err}, + {act: model.ActionAddColumn, jobIDs: []int64{firstID + 7}, cancelRetErrs: noErrs, cancelState: model.StateWriteReorganization, ddlRetErr: err}, + {act: model.ActionAddColumn, jobIDs: []int64{firstID + 8}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 8)}, cancelState: model.StatePublic, ddlRetErr: err}, } return tests @@ -464,23 +460,8 @@ func (s *testDDLSuite) TestCancelJob(c *C) { c.Assert(ctx.Txn(true).Commit(context.Background()), IsNil) s.checkAddIdx(c, d, dbInfo.ID, tblInfo.ID, idxOrigName, true) - // for dropping index - idxName := []interface{}{model.NewCIStr("idx")} - test = &tests[4] - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropIndex, idxName, &test.cancelState) - c.Check(errors.ErrorStack(checkErr), Equals, "") - test = &tests[5] - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropIndex, idxName, &test.cancelState) - c.Check(errors.ErrorStack(checkErr), Equals, "") - test = &tests[6] - doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionDropIndex, idxName, &test.cancelState) - c.Check(errors.ErrorStack(checkErr), Equals, "") - test = &tests[7] - testDropIndex(c, ctx, d, dbInfo, tblInfo, "idx") - c.Check(errors.ErrorStack(checkErr), Equals, "") - // for add column - test = &tests[8] + test = &tests[4] addingColName := "colA" newColumnDef := &ast.ColumnDef{ @@ -494,17 +475,17 @@ func (s *testDDLSuite) TestCancelJob(c *C) { c.Check(errors.ErrorStack(checkErr), Equals, "") s.checkAddColumn(c, d, dbInfo.ID, tblInfo.ID, addingColName, false) - test = &tests[9] + test = &tests[5] doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) c.Check(errors.ErrorStack(checkErr), Equals, "") s.checkAddColumn(c, d, dbInfo.ID, tblInfo.ID, addingColName, false) - test = &tests[10] + test = &tests[6] doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, model.ActionAddColumn, addColumnArgs, &cancelState) c.Check(errors.ErrorStack(checkErr), Equals, "") s.checkAddColumn(c, d, dbInfo.ID, tblInfo.ID, addingColName, false) - test = &tests[11] + test = &tests[7] testAddColumn(c, ctx, d, dbInfo, tblInfo, addColumnArgs) c.Check(errors.ErrorStack(checkErr), Equals, "") s.checkAddColumn(c, d, dbInfo.ID, tblInfo.ID, addingColName, true) From 5ab183f5208bd5d52e5f92bfb3f043f8dd036278 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 30 Nov 2018 22:15:32 +0800 Subject: [PATCH 04/10] address comment --- ddl/db_test.go | 50 +++++++++++++++++++++++---------------------- ddl/rollingback.go | 8 ++++---- util/admin/admin.go | 22 +++++++++++++------- 3 files changed, 45 insertions(+), 35 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index fd8315e533f75..6a79b48e803a2 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -548,33 +548,35 @@ func (s *testDBSuite) TestCancelDropIndex(c *C) { defer func() { ddl.ReorgWaitTimeout = oldReorgWaitTimeout }() hook := &ddl.TestDDLCallback{} var jobID int64 - for _, testCase := range testCases { - hook.OnJobRunBeforeExported = func(job *model.Job) { - if job.Type == model.ActionDropIndex && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobID = job.ID - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = s.store - err := hookCtx.NewTxn() - if err != nil { - checkErr = errors.Trace(err) - return - } - errs, err := admin.CancelJobs(hookCtx.Txn(true), jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } + testCase := &testCases[0] + hook.OnJobRunBeforeExported = func(job *model.Job) { + if job.Type == model.ActionDropIndex && job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { + jobID = job.ID + jobIDs := []int64{job.ID} + hookCtx := mock.NewContext() + hookCtx.Store = s.store + err := hookCtx.NewTxn() + if err != nil { + checkErr = errors.Trace(err) + return + } + errs, err := admin.CancelJobs(hookCtx.Txn(true), jobIDs) + if err != nil { + checkErr = errors.Trace(err) + return + } - checkErr = hookCtx.Txn(true).Commit(context.Background()) + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return } + + checkErr = hookCtx.Txn(true).Commit(context.Background()) } - s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + } + s.dom.DDL().(ddl.DDLForTest).SetHook(hook) + for i := range testCases { + testCase = &testCases[i] if testCase.needAddIndex { s.mustExec(c, "alter table t add index idx_c2(c2)") } diff --git a/ddl/rollingback.go b/ddl/rollingback.go index d286a3cd2f3ef..0fabf76aef165 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -141,8 +141,10 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { originalState := indexInfo.State switch indexInfo.State { case model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: - // we can not rollback now, so just continue to drop index. + // We can not rollback now, so just continue to drop index. + // Normally won't fetch here, because there is check when cancel ddl jobs. see function: isJobRollbackable. job.State = model.JobStateRunning + return ver, nil case model.StatePublic, model.StateWriteOnly: job.State = model.JobStateRollbackDone indexInfo.State = model.StatePublic @@ -156,9 +158,7 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { if err != nil { return ver, errors.Trace(err) } - if job.State == model.JobStateRollbackDone { - job.FinishTableJob(model.JobStateRollbackDone, model.StatePublic, ver, tblInfo) - } + job.FinishTableJob(model.JobStateRollbackDone, model.StatePublic, ver, tblInfo) return ver, errCancelledDDLJob } diff --git a/util/admin/admin.go b/util/admin/admin.go index 3e5fa0e8c56c4..4dd050f63e012 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -83,6 +83,17 @@ func GetDDLInfo(txn kv.Transaction) (*DDLInfo, error) { return info, nil } +func isJobRollbackable(job *model.Job, id int64) error { + switch job.Type { + case model.ActionDropIndex: + if job.SchemaState == model.StateDeleteOnly || + job.SchemaState == model.StateDeleteReorganization { + return ErrCannotCancelDDLJob.GenWithStackByArgs(id) + } + } + return nil +} + // CancelJobs cancels the DDL jobs. func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { if len(ids) == 0 { @@ -113,14 +124,11 @@ func CancelJobs(txn kv.Transaction, ids []int64) ([]error, error) { if job.IsCancelled() || job.IsRollingback() || job.IsRollbackDone() { continue } - switch job.Type { - case model.ActionDropIndex: - if job.SchemaState == model.StateDeleteOnly || - job.SchemaState == model.StateDeleteReorganization { - errs[i] = ErrCannotCancelDDLJob.GenWithStackByArgs(id) - continue - } + errs[i] = isJobRollbackable(job, id) + if errs[i] != nil { + continue } + job.State = model.JobStateCancelling // Make sure RawArgs isn't overwritten. err := job.DecodeArgs(job.RawArgs) From 69c0d67117564912728be3e7ce3914dd34f19485 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 30 Nov 2018 22:30:02 +0800 Subject: [PATCH 05/10] refine code --- ddl/db_test.go | 3 ++- ddl/ddl_api.go | 5 +++-- ddl/index.go | 25 ++++++------------------- ddl/rollingback.go | 5 +++-- planner/core/planbuilder.go | 12 ++---------- util/schemautil/schemautil.go | 26 ++++++++++++++++++++++++++ 6 files changed, 42 insertions(+), 34 deletions(-) create mode 100644 util/schemautil/schemautil.go diff --git a/ddl/db_test.go b/ddl/db_test.go index 6a79b48e803a2..98d13729d32ee 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -15,6 +15,7 @@ package ddl_test import ( "fmt" + "github.com/pingcap/tidb/util/schemautil" "io" "math" "math/rand" @@ -586,7 +587,7 @@ func (s *testDBSuite) TestCancelDropIndex(c *C) { } t := s.testGetTable(c, "t") - indexInfo := ddl.FindIndexByName("idx_c2", t.Meta().Indices) + indexInfo := schemautil.FindIndexByName("idx_c2", t.Meta().Indices) if testCase.cancelSucc { c.Assert(checkErr, IsNil) c.Assert(err, NotNil) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2789be24cf8ee..5dbc450d315f2 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -20,6 +20,7 @@ package ddl import ( "bytes" "fmt" + "github.com/pingcap/tidb/util/schemautil" "strings" "time" @@ -1994,7 +1995,7 @@ func (d *ddl) CreateIndex(ctx sessionctx.Context, ti ast.Ident, unique bool, ind indexName = getAnonymousIndex(t, idxColNames[0].Column.Name) } - if indexInfo := findIndexByName(indexName.L, t.Meta().Indices); indexInfo != nil { + if indexInfo := schemautil.FindIndexByName(indexName.L, t.Meta().Indices); indexInfo != nil { return ErrDupKeyName.GenWithStack("index already exist %s", indexName) } @@ -2119,7 +2120,7 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) } - if indexInfo := findIndexByName(indexName.L, t.Meta().Indices); indexInfo == nil { + if indexInfo := schemautil.FindIndexByName(indexName.L, t.Meta().Indices); indexInfo == nil { return ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } diff --git a/ddl/index.go b/ddl/index.go index cc081fe288078..9a00e181d0bb0 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -15,6 +15,7 @@ package ddl import ( "context" + "github.com/pingcap/tidb/util/schemautil" "math" "sync/atomic" "time" @@ -182,7 +183,7 @@ func dropIndexColumnFlag(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) { } func validateRenameIndex(from, to model.CIStr, tbl *model.TableInfo) (ignore bool, err error) { - if fromIdx := findIndexByName(from.L, tbl.Indices); fromIdx == nil { + if fromIdx := schemautil.FindIndexByName(from.L, tbl.Indices); fromIdx == nil { return false, errors.Trace(infoschema.ErrKeyNotExists.GenWithStackByArgs(from.O, tbl.Name)) } // Take case-sensitivity into account, if `FromKey` and `ToKey` are the same, nothing need to be changed @@ -192,7 +193,7 @@ func validateRenameIndex(from, to model.CIStr, tbl *model.TableInfo) (ignore boo // If spec.FromKey.L == spec.ToKey.L, we operate on the same index(case-insensitive) and change its name (case-sensitive) // e.g: from `inDex` to `IndEX`. Otherwise, we try to rename an index to another different index which already exists, // that's illegal by rule. - if toIdx := findIndexByName(to.L, tbl.Indices); toIdx != nil && from.L != to.L { + if toIdx := schemautil.FindIndexByName(to.L, tbl.Indices); toIdx != nil && from.L != to.L { return false, errors.Trace(infoschema.ErrKeyNameDuplicate.GenWithStackByArgs(toIdx.Name.O)) } return false, nil @@ -219,7 +220,7 @@ func onRenameIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - idx := findIndexByName(from.L, tblInfo.Indices) + idx := schemautil.FindIndexByName(from.L, tblInfo.Indices) idx.Name = to if ver, err = updateVersionAndTableInfo(t, job, tblInfo, true); err != nil { job.State = model.JobStateCancelled @@ -258,7 +259,7 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int return ver, errors.Trace(err) } - indexInfo := findIndexByName(indexName.L, tblInfo.Indices) + indexInfo := schemautil.FindIndexByName(indexName.L, tblInfo.Indices) if indexInfo != nil && indexInfo.State == model.StatePublic { job.State = model.JobStateCancelled return ver, ErrDupKeyName.GenWithStack("index already exist %s", indexName) @@ -371,7 +372,7 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } - indexInfo := findIndexByName(indexName.L, tblInfo.Indices) + indexInfo := schemautil.FindIndexByName(indexName.L, tblInfo.Indices) if indexInfo == nil { job.State = model.JobStateCancelled return ver, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) @@ -1170,20 +1171,6 @@ func findNextPartitionID(currentPartition int64, defs []model.PartitionDefinitio return 0, errors.Errorf("partition id not found %d", currentPartition) } -func findIndexByName(idxName string, indices []*model.IndexInfo) *model.IndexInfo { - for _, idx := range indices { - if idx.Name.L == idxName { - return idx - } - } - return nil -} - -// FindIndexByName exports for testing. -func FindIndexByName(idxName string, indices []*model.IndexInfo) *model.IndexInfo { - return findIndexByName(idxName, indices) -} - func allocateIndexID(tblInfo *model.TableInfo) int64 { tblInfo.MaxIndexID++ return tblInfo.MaxIndexID diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 0fabf76aef165..ffb0690084928 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/util/schemautil" log "github.com/sirupsen/logrus" ) @@ -67,7 +68,7 @@ func convertNotStartAddIdxJob2RollbackJob(t *meta.Meta, job *model.Job, occuredE return ver, errors.Trace(err) } - indexInfo := findIndexByName(indexName.L, tblInfo.Indices) + indexInfo := schemautil.FindIndexByName(indexName.L, tblInfo.Indices) if indexInfo == nil { job.State = model.JobStateCancelled return ver, errCancelledDDLJob @@ -132,7 +133,7 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, errors.Trace(err) } - indexInfo := findIndexByName(indexName.L, tblInfo.Indices) + indexInfo := schemautil.FindIndexByName(indexName.L, tblInfo.Indices) if indexInfo == nil { job.State = model.JobStateCancelled return ver, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index e37532fe500f2..c2e46a4ac582f 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -15,6 +15,7 @@ package core import ( "fmt" + "github.com/pingcap/tidb/util/schemautil" "strings" "github.com/cznic/mathutil" @@ -383,15 +384,6 @@ func removeIgnoredPaths(paths, ignoredPaths []*accessPath, tblInfo *model.TableI return remainedPaths } -func findIndexByName(indices []*model.IndexInfo, name model.CIStr) *model.IndexInfo { - for _, idx := range indices { - if idx.Name.L == name.L { - return idx - } - } - return nil -} - func (b *PlanBuilder) buildSelectLock(src LogicalPlan, lock ast.SelectLockType) *LogicalLock { selectLock := LogicalLock{Lock: lock}.Init(b.ctx) selectLock.SetChildren(src) @@ -723,7 +715,7 @@ func (b *PlanBuilder) buildAnalyzeIndex(as *ast.AnalyzeTableStmt) (Plan, error) return nil, err } for _, idxName := range as.IndexNames { - idx := findIndexByName(tblInfo.Indices, idxName) + idx := schemautil.FindIndexByName(idxName.L, tblInfo.Indices) if idx == nil || idx.State != model.StatePublic { return nil, ErrAnalyzeMissIndex.GenWithStackByArgs(idxName.O, tblInfo.Name.O) } diff --git a/util/schemautil/schemautil.go b/util/schemautil/schemautil.go new file mode 100644 index 0000000000000..6a12b5ead734d --- /dev/null +++ b/util/schemautil/schemautil.go @@ -0,0 +1,26 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package schemautil + +import "github.com/pingcap/parser/model" + +// FindIndexByName finds index by name. +func FindIndexByName(idxName string, indices []*model.IndexInfo) *model.IndexInfo { + for _, idx := range indices { + if idx.Name.L == idxName { + return idx + } + } + return nil +} From 594b5a412c2bee89719a33045713bd4ee5fc1b5a Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Mon, 3 Dec 2018 13:38:52 +0800 Subject: [PATCH 06/10] address comment --- ddl/db_test.go | 2 +- ddl/ddl_api.go | 2 +- ddl/index.go | 2 +- planner/core/planbuilder.go | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 98d13729d32ee..c3e2df58fed4d 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -15,7 +15,6 @@ package ddl_test import ( "fmt" - "github.com/pingcap/tidb/util/schemautil" "io" "math" "math/rand" @@ -47,6 +46,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" "github.com/pingcap/tidb/util/mock" + "github.com/pingcap/tidb/util/schemautil" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" "github.com/pingcap/tidb/util/testutil" diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 5dbc450d315f2..8169bd8e5becf 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -20,7 +20,6 @@ package ddl import ( "bytes" "fmt" - "github.com/pingcap/tidb/util/schemautil" "strings" "time" @@ -37,6 +36,7 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/schemautil" ) func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetInfo *ast.CharsetOpt) (err error) { diff --git a/ddl/index.go b/ddl/index.go index 9a00e181d0bb0..3148c7bc4bcd1 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -15,7 +15,6 @@ package ddl import ( "context" - "github.com/pingcap/tidb/util/schemautil" "math" "sync/atomic" "time" @@ -38,6 +37,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/rowDecoder" + "github.com/pingcap/tidb/util/schemautil" "github.com/pingcap/tidb/util/timeutil" log "github.com/sirupsen/logrus" ) diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index c2e46a4ac582f..2d34e36c1fc19 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -15,7 +15,6 @@ package core import ( "fmt" - "github.com/pingcap/tidb/util/schemautil" "strings" "github.com/cznic/mathutil" @@ -34,6 +33,7 @@ import ( "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util/ranger" + "github.com/pingcap/tidb/util/schemautil" ) type visitInfo struct { From fd9864d39ebbf5ff17dbe3702cb9e3db318f6096 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Wed, 5 Dec 2018 20:11:45 +0800 Subject: [PATCH 07/10] address comment --- util/admin/admin.go | 1 + 1 file changed, 1 insertion(+) diff --git a/util/admin/admin.go b/util/admin/admin.go index 4dd050f63e012..27838ca2cadc2 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -86,6 +86,7 @@ func GetDDLInfo(txn kv.Transaction) (*DDLInfo, error) { func isJobRollbackable(job *model.Job, id int64) error { switch job.Type { case model.ActionDropIndex: + // We can't cancel if index current state is in StateDeleteOnly or StateDeleteReorganization, otherwise will cause inconsistent between record and index. if job.SchemaState == model.StateDeleteOnly || job.SchemaState == model.StateDeleteReorganization { return ErrCannotCancelDDLJob.GenWithStackByArgs(id) From 283742bf77c85668f45136f40bede2554e7ea863 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 6 Dec 2018 13:10:53 +0800 Subject: [PATCH 08/10] address comment --- ddl/rollingback.go | 1 - 1 file changed, 1 deletion(-) diff --git a/ddl/rollingback.go b/ddl/rollingback.go index ffb0690084928..99143555c115f 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -129,7 +129,6 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { schemaID := job.SchemaID tblInfo, err := getTableInfo(t, job, schemaID) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } From c7398038e1f15661792e4f49569ff63ce472a197 Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Thu, 6 Dec 2018 14:55:12 +0800 Subject: [PATCH 09/10] refine code --- ddl/rollingback.go | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 99143555c115f..db8da52003162 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -119,16 +119,16 @@ func rollingbackAddColumn(t *meta.Meta, job *model.Job) (ver int64, err error) { } func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { - var indexName model.CIStr - err = job.DecodeArgs(&indexName) + schemaID := job.SchemaID + tblInfo, err := getTableInfo(t, job, schemaID) if err != nil { - job.State = model.JobStateCancelled return ver, errors.Trace(err) } - schemaID := job.SchemaID - tblInfo, err := getTableInfo(t, job, schemaID) + var indexName model.CIStr + err = job.DecodeArgs(&indexName) if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } From a43f677d0f2e622e89eca7d5b6797bd0da50b36e Mon Sep 17 00:00:00 2001 From: crazycs520 Date: Fri, 7 Dec 2018 20:29:40 +0800 Subject: [PATCH 10/10] fix test --- ddl/db_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index ba4babc465539..1406e0b639370 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -556,7 +556,7 @@ func (s *testDBSuite) TestCancelDropIndex(c *C) { jobIDs := []int64{job.ID} hookCtx := mock.NewContext() hookCtx.Store = s.store - err := hookCtx.NewTxn() + err := hookCtx.NewTxn(context.TODO()) if err != nil { checkErr = errors.Trace(err) return