From 0483c4c7f6b855b5cd8e7d5c8e4784873db21238 Mon Sep 17 00:00:00 2001 From: tangenta Date: Tue, 5 Jul 2022 16:33:02 +0800 Subject: [PATCH] ddl: support drop indexes for multi-schema change (#35883) ref pingcap/tidb#14766 --- ddl/cancel_test.go | 16 +++--- ddl/ddl_api.go | 16 ++---- ddl/ddl_worker_test.go | 2 +- ddl/delete_range.go | 6 ++- ddl/index.go | 87 ++++++++++++++++++++++----------- ddl/index_modify_test.go | 17 ++++--- ddl/multi_schema_change.go | 3 ++ ddl/multi_schema_change_test.go | 83 +++++++++++++++++++++++++++++++ ddl/rollingback.go | 6 +-- ddl/sanity_check.go | 6 ++- 10 files changed, 178 insertions(+), 64 deletions(-) diff --git a/ddl/cancel_test.go b/ddl/cancel_test.go index 9f367eb40ca55..64179c19a8eca 100644 --- a/ddl/cancel_test.go +++ b/ddl/cancel_test.go @@ -206,14 +206,13 @@ var allTestCase = []testCancelJob{ {"alter table t_partition drop partition p6", false, model.StateDeleteReorganization, true, true, []string{"alter table t_partition add partition (partition p6 values less than (8192))"}}, {"alter table t_partition drop partition p6", false, model.StateNone, true, true, []string{"alter table t_partition add partition (partition p6 values less than (8192))"}}, // Drop indexes. - // TODO: fix schema state. - {"alter table t drop index mul_idx1, drop index mul_idx2", true, model.StateNone, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateWriteOnly, true, false, nil}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateWriteOnly, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateDeleteOnly, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateDeleteOnly, false, true, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateDeleteReorganization, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, - {"alter table t drop index mul_idx1, drop index mul_idx2", false, model.StateDeleteReorganization, false, true, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, + {"alter table t drop index mul_idx1, drop index mul_idx2", true, subStates{model.StatePublic, model.StatePublic}, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateWriteOnly, model.StateWriteOnly}, true, false, nil}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateWriteOnly, model.StateWriteOnly}, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateDeleteOnly, model.StateWriteOnly}, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateDeleteOnly, model.StateWriteOnly}, false, true, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateDeleteReorganization, model.StateWriteOnly}, true, false, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, + {"alter table t drop index mul_idx1, drop index mul_idx2", false, subStates{model.StateDeleteReorganization, model.StateWriteOnly}, false, true, []string{"alter table t add index mul_idx1(c1)", "alter table t add index mul_idx2(c1)"}}, // Alter db placement. {"alter database db_placement placement policy = 'alter_x'", true, model.StateNone, true, false, []string{"create placement policy alter_x PRIMARY_REGION=\"cn-east-1\", REGIONS=\"cn-east-1\";", "create database db_placement"}}, {"alter database db_placement placement policy = 'alter_x'", false, model.StatePublic, false, true, nil}, @@ -304,7 +303,6 @@ func TestCancel(t *testing.T) { for _, prepareSQL := range tc.prepareSQL { tk.MustExec(prepareSQL) } - cancel = false cancelWhenReorgNotStart = true registHook(hook, true) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d1c92e48eedb1..cbf71b40b3df3 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -3078,7 +3078,7 @@ func checkMultiSpecs(sctx sessionctx.Context, specs []*ast.AlterTableSpec) error func allSupported(specs []*ast.AlterTableSpec) bool { for _, s := range specs { switch s.Tp { - case ast.AlterTableAddColumns, ast.AlterTableDropColumn: + case ast.AlterTableAddColumns, ast.AlterTableDropColumn, ast.AlterTableDropIndex, ast.AlterTableDropPrimaryKey: default: return false } @@ -3118,10 +3118,9 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast if len(validSpecs) > 1 { useMultiSchemaChange := false switch validSpecs[0].Tp { - case ast.AlterTableAddColumns, ast.AlterTableDropColumn: + case ast.AlterTableAddColumns, ast.AlterTableDropColumn, + ast.AlterTableDropPrimaryKey, ast.AlterTableDropIndex: useMultiSchemaChange = true - case ast.AlterTableDropPrimaryKey, ast.AlterTableDropIndex: - err = d.DropIndexes(sctx, ident, validSpecs) default: return dbterror.ErrRunMultiSchemaChanges } @@ -6124,19 +6123,14 @@ func (d *ddl) dropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI SchemaID: schema.ID, TableID: t.Meta().ID, SchemaName: schema.Name.L, + SchemaState: indexInfo.State, TableName: t.Meta().Name.L, Type: jobTp, BinlogInfo: &model.HistoryInfo{}, - SchemaState: indexInfo.State, - Args: []interface{}{indexName}, + Args: []interface{}{indexName, ifExists}, } err = d.DoDDLJob(ctx, job) - // index not exists, but if_exists flags is true, so we ignore this error. - if dbterror.ErrCantDropFieldOrKey.Equal(err) && ifExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil - } err = d.callHookOnChanged(job, err) return errors.Trace(err) } diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 379c2b34df24a..6c21be950aead 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -11,7 +11,7 @@ // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. // See the License for the specific language governing permissions and // limitations under the License. -// + package ddl_test import ( diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 636c5d54edb2e..93b5e6cad4ba8 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -324,8 +324,9 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, sctx sessionctx.Context, case model.ActionAddIndex, model.ActionAddPrimaryKey: tableID := job.TableID var indexID int64 + var ifExists bool var partitionIDs []int64 - if err := job.DecodeArgs(&indexID, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexID, &ifExists, &partitionIDs); err != nil { return errors.Trace(err) } if len(partitionIDs) > 0 { @@ -346,9 +347,10 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, sctx sessionctx.Context, case model.ActionDropIndex, model.ActionDropPrimaryKey: tableID := job.TableID var indexName interface{} + var ifExists bool var indexID int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexName, &indexID, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexName, &ifExists, &indexID, &partitionIDs); err != nil { return errors.Trace(err) } if len(partitionIDs) > 0 { diff --git a/ddl/index.go b/ddl/index.go index 6701a1e905d32..da5eb3418278b 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -46,6 +46,7 @@ import ( "github.com/tikv/client-go/v2/oracle" "github.com/tikv/client-go/v2/tikv" "go.uber.org/zap" + "golang.org/x/exp/slices" ) const ( @@ -634,19 +635,23 @@ func doReorgWorkForCreateIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Jo } func onDropIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - tblInfo, indexInfo, err := checkDropIndex(t, job) + tblInfo, indexInfo, ifExists, err := checkDropIndex(t, job) if err != nil { + if ifExists && dbterror.ErrCantDropFieldOrKey.Equal(err) { + job.Warning = toTError(err) + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + return ver, nil + } return ver, errors.Trace(err) } if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { return ver, errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Drop Index")) } - dependentHiddenCols := make([]*model.ColumnInfo, 0) - for _, indexColumn := range indexInfo.Columns { - if tblInfo.Columns[indexColumn.Offset].Hidden { - dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) - } + if job.MultiSchemaInfo != nil && !job.IsRollingback() && job.MultiSchemaInfo.Revertible { + job.MarkNonRevertible() + job.SchemaState = indexInfo.State + return updateVersionAndTableInfo(d, t, job, tblInfo, false) } originalState := indexInfo.State @@ -675,24 +680,11 @@ func onDropIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { case model.StateDeleteReorganization: // reorganization -> absent indexInfo.State = model.StateNone - if len(dependentHiddenCols) > 0 { - firstHiddenOffset := dependentHiddenCols[0].Offset - for i := 0; i < len(dependentHiddenCols); i++ { - // Set this column's offset to the last and reset all following columns' offsets. - adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) - } - } - newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if idx.Name.L != indexInfo.Name.L { - newIndices = append(newIndices, idx) - } - } - tblInfo.Indices = newIndices // Set column index flag. dropIndexColumnFlag(tblInfo, indexInfo) + removeDependentHiddenColumns(tblInfo, indexInfo) + removeIndexInfo(tblInfo, indexInfo) - tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(dependentHiddenCols)] failpoint.Inject("mockExceedErrorLimit", func(val failpoint.Value) { if val.(bool) { panic("panic test in cancelling add index") @@ -721,38 +713,75 @@ func onDropIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } -func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.IndexInfo, error) { +func removeDependentHiddenColumns(tblInfo *model.TableInfo, idxInfo *model.IndexInfo) { + hiddenColOffs := make([]int, 0) + for _, indexColumn := range idxInfo.Columns { + col := tblInfo.Columns[indexColumn.Offset] + if col.Hidden { + hiddenColOffs = append(hiddenColOffs, col.Offset) + } + } + // Sort the offset in descending order. + slices.SortFunc(hiddenColOffs, func(a, b int) bool { return a > b }) + // Move all the dependent hidden columns to the end. + endOffset := len(tblInfo.Columns) - 1 + for _, offset := range hiddenColOffs { + tblInfo.MoveColumnInfo(offset, endOffset) + } + tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(hiddenColOffs)] +} + +func removeIndexInfo(tblInfo *model.TableInfo, idxInfo *model.IndexInfo) { + indices := tblInfo.Indices + offset := -1 + for i, idx := range indices { + if idxInfo.ID == idx.ID { + offset = i + break + } + } + if offset == -1 { + // The target index has been removed. + return + } + // Remove the target index. + tblInfo.Indices = append(tblInfo.Indices[:offset], tblInfo.Indices[offset+1:]...) +} + +func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.IndexInfo, bool /* ifExists */, error) { schemaID := job.SchemaID tblInfo, err := GetTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { - return nil, nil, errors.Trace(err) + return nil, nil, false, errors.Trace(err) } var indexName model.CIStr - if err = job.DecodeArgs(&indexName); err != nil { + var ifExists bool + if err = job.DecodeArgs(&indexName, &ifExists); err != nil { job.State = model.JobStateCancelled - return nil, nil, errors.Trace(err) + return nil, nil, false, errors.Trace(err) } indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { job.State = model.JobStateCancelled - return nil, nil, dbterror.ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + return nil, nil, ifExists, dbterror.ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } // Double check for drop index on auto_increment column. err = checkDropIndexOnAutoIncrementColumn(tblInfo, indexInfo) if err != nil { job.State = model.JobStateCancelled - return nil, nil, autoid.ErrWrongAutoKey + return nil, nil, false, autoid.ErrWrongAutoKey } // Check that drop primary index will not cause invisible implicit primary index. if err := checkInvisibleIndexesOnPK(tblInfo, []*model.IndexInfo{indexInfo}, job); err != nil { - return nil, nil, errors.Trace(err) + job.State = model.JobStateCancelled + return nil, nil, false, errors.Trace(err) } - return tblInfo, indexInfo, nil + return tblInfo, indexInfo, false, nil } func onDropIndexes(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { diff --git a/ddl/index_modify_test.go b/ddl/index_modify_test.go index 13e423f5b9b66..bec851c44d450 100644 --- a/ddl/index_modify_test.go +++ b/ddl/index_modify_test.go @@ -886,16 +886,18 @@ func testDropIndexesIfExists(t *testing.T, store kv.Storage) { tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Note|1091|index i3 doesn't exist")) // Verify the impact of deletion order when dropping duplicate indexes. - tk.MustGetErrMsg( + tk.MustGetErrCode( "alter table test_drop_indexes_if_exists drop index i2, drop index i2;", - "[ddl:1091]index i2 doesn't exist", + errno.ErrUnsupportedDDLOperation, ) - tk.MustGetErrMsg( + tk.MustGetErrCode( "alter table test_drop_indexes_if_exists drop index if exists i2, drop index i2;", - "[ddl:1091]index i2 doesn't exist", + errno.ErrUnsupportedDDLOperation, + ) + tk.MustGetErrCode( + "alter table test_drop_indexes_if_exists drop index i2, drop index if exists i2;", + errno.ErrUnsupportedDDLOperation, ) - tk.MustExec("alter table test_drop_indexes_if_exists drop index i2, drop index if exists i2;") - tk.MustQuery("show warnings;").Check(testkit.RowsWithSep("|", "Note|1091|index i2 doesn't exist")) } func testDropIndexesFromPartitionedTable(t *testing.T, store kv.Storage) { @@ -911,7 +913,8 @@ func testDropIndexesFromPartitionedTable(t *testing.T, store kv.Storage) { } tk.MustExec("alter table test_drop_indexes_from_partitioned_table drop index i1, drop index if exists i2;") tk.MustExec("alter table test_drop_indexes_from_partitioned_table add index i1(c1)") - tk.MustExec("alter table test_drop_indexes_from_partitioned_table drop index i1, drop index if exists i1;") + tk.MustGetErrCode("alter table test_drop_indexes_from_partitioned_table drop index i1, drop index if exists i1;", + errno.ErrUnsupportedDDLOperation) tk.MustExec("alter table test_drop_indexes_from_partitioned_table drop column c1, drop column c2;") tk.MustExec("alter table test_drop_indexes_from_partitioned_table add column c1 int") tk.MustGetErrCode("alter table test_drop_indexes_from_partitioned_table drop column c1, drop column if exists c1;", diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index ca1e60056fae8..20e62d89ed22b 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -186,6 +186,9 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error case model.ActionDropColumn: colName := job.Args[0].(model.CIStr) info.DropColumns = append(info.DropColumns, colName) + case model.ActionDropIndex, model.ActionDropPrimaryKey: + indexName := job.Args[0].(model.CIStr) + info.DropIndexes = append(info.DropIndexes, indexName) default: return dbterror.ErrRunMultiSchemaChanges } diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index acc778258834a..d52af494bad79 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -241,6 +241,89 @@ func TestMultiSchemaChangeDropColumnsParallel(t *testing.T) { }) } +func TestMultiSchemaChangeDropIndexes(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + + // Test drop same index. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int, index t(a));") + tk.MustGetErrCode("alter table t drop index t, drop index t", errno.ErrUnsupportedDDLOperation) + + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (id int, c1 int, c2 int, primary key(id) nonclustered, key i1(c1), key i2(c2), key i3(c1, c2));") + tk.MustExec("insert into t values (1, 2, 3);") + tk.MustExec("alter table t drop index i1, drop index i2;") + tk.MustGetErrCode("select * from t use index(i1);", errno.ErrKeyDoesNotExist) + tk.MustGetErrCode("select * from t use index(i2);", errno.ErrKeyDoesNotExist) + tk.MustExec("alter table t drop index i3, drop primary key;") + tk.MustGetErrCode("select * from t use index(primary);", errno.ErrKeyDoesNotExist) + tk.MustGetErrCode("select * from t use index(i3);", errno.ErrKeyDoesNotExist) + + // Test drop index with drop column. + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (a int default 1, b int default 2, c int default 3, index t(a))") + tk.MustExec("insert into t values ();") + tk.MustExec("alter table t drop index t, drop column a") + tk.MustGetErrCode("select * from t force index(t)", errno.ErrKeyDoesNotExist) +} + +func TestMultiSchemaChangeDropIndexesCancelled(t *testing.T) { + store, dom, clean := testkit.CreateMockStoreAndDomain(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + originHook := dom.DDL().GetHook() + + // Test for cancelling the job in a middle state. + tk.MustExec("create table t (a int, b int, index(a), unique index(b), index idx(a, b));") + hook := newCancelJobHook(store, dom, func(job *model.Job) bool { + return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StateDeleteOnly + }) + dom.DDL().SetHook(hook) + tk.MustExec("alter table t drop index a, drop index b, drop index idx;") + dom.DDL().SetHook(originHook) + hook.MustCancelFailed(t) + tk.MustGetErrCode("select * from t use index (a);", errno.ErrKeyDoesNotExist) + tk.MustGetErrCode("select * from t use index (b);", errno.ErrKeyDoesNotExist) + tk.MustGetErrCode("select * from t use index (idx);", errno.ErrKeyDoesNotExist) + + // Test for cancelling the job in none state. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, index(a), unique index(b), index idx(a, b));") + hook = newCancelJobHook(store, dom, func(job *model.Job) bool { + return job.MultiSchemaInfo.SubJobs[1].SchemaState == model.StatePublic + }) + dom.DDL().SetHook(hook) + tk.MustGetErrCode("alter table t drop index a, drop index b, drop index idx;", errno.ErrCancelledDDLJob) + dom.DDL().SetHook(originHook) + hook.MustCancelDone(t) + tk.MustQuery("select * from t use index (a);").Check(testkit.Rows()) + tk.MustQuery("select * from t use index (b);").Check(testkit.Rows()) + tk.MustQuery("select * from t use index (idx);").Check(testkit.Rows()) +} + +func TestMultiSchemaChangeDropIndexesParallel(t *testing.T) { + store, clean := testkit.CreateMockStore(t) + defer clean() + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (a int, b int, c int, index(a), index(b), index(c));") + putTheSameDDLJobTwice(t, func() { + tk.MustExec("alter table t drop index if exists b, drop index if exists c;") + tk.MustQuery("show warnings").Check(testkit.Rows( + "Note 1091 index b doesn't exist", + "Note 1091 index c doesn't exist")) + }) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int, b int, c int, index (a), index(b), index(c));") + putTheSameDDLJobTwice(t, func() { + tk.MustGetErrCode("alter table t drop index b, drop index a;", errno.ErrCantDropFieldOrKey) + }) +} + type cancelOnceHook struct { store kv.Storage triggered bool diff --git a/ddl/rollingback.go b/ddl/rollingback.go index bcb89b591c796..2b9ea91cd3716 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -60,8 +60,8 @@ func convertAddIdxJob2RollbackJob(d *ddlCtx, t *meta.Meta, job *model.Job, tblIn } } - // the second args will be used in onDropIndex. - job.Args = []interface{}{indexInfo.Name, getPartitionIDs(tblInfo)} + // the second and the third args will be used in onDropIndex. + job.Args = []interface{}{indexInfo.Name, false /* ifExists */, getPartitionIDs(tblInfo)} // If add index job rollbacks in write reorganization state, its need to delete all keys which has been added. // Its work is the same as drop index job do. // The write reorganization state in add index job that likes write only state in drop index job. @@ -208,7 +208,7 @@ func rollingbackDropColumn(t *meta.Meta, job *model.Job) (ver int64, err error) } func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { - _, indexInfo, err := checkDropIndex(t, job) + _, indexInfo, _, err := checkDropIndex(t, job) if err != nil { return ver, errors.Trace(err) } diff --git a/ddl/sanity_check.go b/ddl/sanity_check.go index e81b9f31b5c5a..6a62e0695e003 100644 --- a/ddl/sanity_check.go +++ b/ddl/sanity_check.go @@ -102,16 +102,18 @@ func expectedDeleteRangeCnt(job *model.Job) (int, error) { return len(physicalTableIDs), nil case model.ActionAddIndex, model.ActionAddPrimaryKey: var indexID int64 + var ifExists bool var partitionIDs []int64 - if err := job.DecodeArgs(&indexID, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexID, &ifExists, &partitionIDs); err != nil { return 0, errors.Trace(err) } return mathutil.Max(len(partitionIDs), 1), nil case model.ActionDropIndex, model.ActionDropPrimaryKey: var indexName interface{} + var ifNotExists bool var indexID int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexName, &indexID, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexName, &ifNotExists, &indexID, &partitionIDs); err != nil { return 0, errors.Trace(err) } return mathutil.Max(len(partitionIDs), 1), nil