From 2f127517f24daac89774da50081d8cdee2cdf61d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 14 Oct 2020 21:19:27 +0800 Subject: [PATCH 01/61] =?UTF-8?q?add=20func=EF=BC=9ADropIndexes?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- ddl/ddl_api.go | 88 +++++++++++++++++++++++++++++++++++++++++++++++++- 1 file changed, 87 insertions(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 232ece09ec22e..58b74cee33833 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4789,7 +4789,6 @@ func (d *ddl) CreateForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName mode err = d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) return errors.Trace(err) - } func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model.CIStr) error { @@ -4891,6 +4890,93 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI return errors.Trace(err) } +func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.AlterTableSpec) error { + is := d.infoHandle.Get() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return errors.Trace(infoschema.ErrDatabaseNotExists) + } + t, err := is.TableByName(ti.Schema, ti.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + + allIndexIsExists := true + indexNames := make([]interface{}, 0, len(specs)) + for _, spec := range specs { + var indexName model.CIStr + switch spec.Tp { + case ast.AlterTableDropIndex: + indexName = model.NewCIStr(spec.Name) + case ast.AlterTableDropPrimaryKey: + indexName = model.NewCIStr(mysql.PrimaryKeyName) + default: + return nil + } + + indexInfo := t.Meta().FindIndexByName(indexName.L) + var isPK bool + if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && + // Before we fixed #14243, there might be a general index named `primary` but not a primary key. + (indexInfo == nil || indexInfo.Primary) { + isPK = true + } + if isPK { + if !config.GetGlobalConfig().AlterPrimaryKey { + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") + } + // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. + if indexInfo == nil && !t.Meta().PKIsHandle { + return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") + } + if t.Meta().PKIsHandle { + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") + } + if t.Meta().IsCommonHandle { + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + } + } + if indexInfo == nil { + err = ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + if spec.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + continue + } + return err + } + + // Check for drop index on auto_increment column. + err = checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo) + if err != nil { + return errors.Trace(err) + } + + indexNames = append(indexNames, indexName) + + if !spec.IfExists { + allIndexIsExists = false + } + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: t.Meta().ID, + SchemaName: schema.Name.L, + Type: model.ActionDropIndex, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{indexNames}, + } + + err = d.doDDLJob(ctx, job) + // index not exists, but if_exists flags is true, so we ignore this error. + if ErrCantDropFieldOrKey.Equal(err) && allIndexIsExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return nil + } + err = d.callHookOnChanged(err) + return errors.Trace(err) +} + func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { // Check whether there are other columns depend on this column or not. for _, col := range tblInfo.Columns { From 59af45cd193a3850d3ae6a891ee861134ee38ad3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 14 Oct 2020 21:19:36 +0800 Subject: [PATCH 02/61] add func: onDropIndexes --- ddl/index.go | 148 +++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 148 insertions(+) diff --git a/ddl/index.go b/ddl/index.go index 7dfb922045896..e27a999f83f54 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -674,6 +674,154 @@ func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } +func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { + tblInfo, indexesInfo, err := checkDropIndexes(t, job) + if err != nil { + return ver, errors.Trace(err) + } + if len(indexesInfo) == 0 { + job.State = model.JobStateCancelled + return ver, nil + } + + dependentHiddenCols := make([]*model.ColumnInfo, 0) + for _, indexInfo := range indexesInfo { + for _, indexColumn := range indexInfo.Columns { + if tblInfo.Columns[indexColumn.Offset].Hidden { + dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) + } + } + } + + indexInfo := indexesInfo[0] + originalState := indexInfo.State + switch indexInfo.State { + case model.StatePublic: + // public -> write only + job.SchemaState = model.StateWriteOnly + for _, indexInfo := range indexesInfo { + indexInfo.State = model.StateWriteOnly + } + if len(dependentHiddenCols) > 0 { + firstHiddenOffset := dependentHiddenCols[0].Offset + for i := 0; i < len(dependentHiddenCols); i++ { + tblInfo.Columns[firstHiddenOffset].State = model.StateWriteOnly + // Set this column's offset to the last and reset all following columns' offsets. + adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) + } + } + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + case model.StateWriteOnly: + // write only -> delete only + job.SchemaState = model.StateDeleteOnly + for _, indexInfo := range indexesInfo { + indexInfo.State = model.StateDeleteOnly + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) + } + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + case model.StateDeleteOnly: + // delete only -> reorganization + job.SchemaState = model.StateDeleteReorganization + for _, indexInfo := range indexesInfo { + indexInfo.State = model.StateDeleteReorganization + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) + } + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + case model.StateDeleteReorganization: + indexesIDs := make([]int64, 0, len(indexesInfo)) + indexesLName := make(map[string]bool, len(indexesInfo)) + for _, indexInfo := range indexesInfo { + indexesLName[indexInfo.Name.L] = true + indexesIDs = append(indexesIDs, indexInfo.ID) + } + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) + for _, idx := range tblInfo.Indices { + if _, ok := indexesLName[idx.Name.L]; !ok { + newIndices = append(newIndices, idx) + } + } + tblInfo.Indices = newIndices + // Set column index flag. + for _, indexInfo := range indexesInfo { + dropIndexColumnFlag(tblInfo, indexInfo) + } + + tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(dependentHiddenCols)] + + ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, originalState != model.StateNone) + if err != nil { + return ver, errors.Trace(err) + } + + // Finish this job. + if job.IsRollingback() { + job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + job.Args[0] = indexesIDs + // the partition ids were append by convertAddIdxJob2RollbackJob, it is weird, but for the compatibility, + // we should keep appending the partitions in the convertAddIdxJob2RollbackJob. + } else { + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + job.Args = append(job.Args, indexesIDs, getPartitionIDs(tblInfo)) + } + default: + err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo.State) + } + return ver, errors.Trace(err) +} + +func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model.IndexInfo, error) { + schemaID := job.SchemaID + tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return nil, nil, errors.Trace(err) + } + + var indexNames []model.CIStr + if err = job.DecodeArgs(&indexNames); err != nil { + job.State = model.JobStateCancelled + return nil, nil, errors.Trace(err) + } + + indexesInfo := make([]*model.IndexInfo, 0, len(indexNames)) + indexesMap := make(map[string]bool) + for _, indexName := range indexNames { + indexInfo := tblInfo.FindIndexByName(indexName.L) + if indexInfo == nil { + job.State = model.JobStateCancelled + return nil, nil, 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 + } + + indexesInfo = append(indexesInfo, indexInfo) + indexesMap[indexName.L] = true + } + + // Check that drop primary index will not cause invisible implicit primary index. + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) + + for _, idx := range tblInfo.Indices { + if _, ok := indexesMap[idx.Name.L]; !ok { + newIndices = append(newIndices, idx) + } + } + + newTbl := tblInfo.Clone() + newTbl.Indices = newIndices + err = checkInvisibleIndexOnPK(newTbl) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, errors.Trace(err) + } + + return tblInfo, indexesInfo, nil +} + func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.IndexInfo, error) { schemaID := job.SchemaID tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID) From 05e91bb1ae6e54b2e985af8a9e2544ab7944dee0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 14 Oct 2020 21:45:37 +0800 Subject: [PATCH 03/61] call func: DropIndexes --- ddl/ddl_api.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 58b74cee33833..7df572b6b66d5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2278,6 +2278,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = d.AddColumns(ctx, ident, validSpecs) case ast.AlterTableDropColumn: err = d.DropColumns(ctx, ident, validSpecs) + case ast.AlterTableDropIndex: + err = d.DropIndexes(ctx, ident, validSpecs) default: return errRunMultiSchemaChanges } From 3fcdc884f36630ca051ddea3bc482f8f3421f86d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 15 Oct 2020 15:49:56 +0800 Subject: [PATCH 04/61] add func: rollingbackDropIndex --- ddl/rollingback.go | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 3aff3ed13f13b..2a05c62fd20a3 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -290,6 +290,43 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { return ver, errCancelledDDLJob } +func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) { + tblInfo, indexesInfo, err := checkDropIndexes(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + indexInfo := indexesInfo[0] + originalState := indexInfo.State + switch indexInfo.State { + case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: + // 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: + job.State = model.JobStateRollbackDone + for _, indexInfo := range indexesInfo { + indexInfo.State = model.StatePublic + } + default: + return ver, ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo.State) + } + + job.SchemaState = indexInfo.State + var indexNames []model.CIStr + for _, indexInfo := range indexesInfo { + indexNames = append(indexNames, indexInfo.Name) + } + job.Args = []interface{}{indexNames} + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + job.FinishTableJob(model.JobStateRollbackDone, model.StatePublic, ver, tblInfo) + return ver, errCancelledDDLJob +} + func rollingbackAddIndex(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job, isPK bool) (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 { From 550a5c1c3d717ccf38aa5da08bb9a2fb0e32f43d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 15 Oct 2020 22:28:05 +0800 Subject: [PATCH 05/61] Replace onDropIndex with onDropIndexes --- ddl/ddl_worker.go | 2 +- ddl/index.go | 10 +++++++--- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index d135e701ac88d..d7b9869b2470e 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -644,7 +644,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionAddPrimaryKey: ver, err = w.onCreateIndex(d, t, job, true) case model.ActionDropIndex, model.ActionDropPrimaryKey: - ver, err = onDropIndex(t, job) + ver, err = onDropIndexes(t, job) case model.ActionRenameIndex: ver, err = onRenameIndex(t, job) case model.ActionAddForeignKey: diff --git a/ddl/index.go b/ddl/index.go index e27a999f83f54..8779cf3a1503c 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -385,7 +385,7 @@ func updateHiddenColumns(tblInfo *model.TableInfo, idxInfo *model.IndexInfo, sta func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK bool) (ver int64, err error) { // Handle the rolling back job. if job.IsRollingback() { - ver, err = onDropIndex(t, job) + ver, err = onDropIndexes(t, job) if err != nil { return ver, errors.Trace(err) } @@ -778,8 +778,12 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. var indexNames []model.CIStr if err = job.DecodeArgs(&indexNames); err != nil { - job.State = model.JobStateCancelled - return nil, nil, errors.Trace(err) + var indexName model.CIStr + if err = job.DecodeArgs(&indexName); err != nil { + job.State = model.JobStateCancelled + return nil, nil, errors.Trace(err) + } + indexNames = append(indexNames, indexName) } indexesInfo := make([]*model.IndexInfo, 0, len(indexNames)) From d8b29d1e0580a739fe0465deaea61c6a836b6c53 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 15 Oct 2020 22:28:47 +0800 Subject: [PATCH 06/61] Replace rollingbackDropIndex with rollingbackDropIndexes --- ddl/rollingback.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 2a05c62fd20a3..9aca28033fa21 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -460,7 +460,7 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) case model.ActionDropColumns: ver, err = rollingbackDropColumns(t, job) case model.ActionDropIndex, model.ActionDropPrimaryKey: - ver, err = rollingbackDropIndex(t, job) + ver, err = rollingbackDropIndexes(t, job) case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: err = rollingbackDropTableOrView(t, job) case model.ActionDropTablePartition: From 0d21ae3d257afd2c96e1fb1ec2aabf1b66382c09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 15 Oct 2020 22:29:33 +0800 Subject: [PATCH 07/61] compatible DropIndex --- ddl/delete_range.go | 35 +++++++++++++++++++++-------------- 1 file changed, 21 insertions(+), 14 deletions(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index a31f24e0a705d..9ea7dc1d631da 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -323,24 +323,31 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error } case model.ActionDropIndex, model.ActionDropPrimaryKey: tableID := job.TableID - var indexName interface{} - var indexID int64 + var indexNames []model.CIStr + var indexIDs []int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexName, &indexID, &partitionIDs); err != nil { - return errors.Trace(err) + if err := job.DecodeArgs(&indexNames, &indexIDs, &partitionIDs); err != nil { + var indexName model.CIStr + if err := job.DecodeArgs(&indexName, &indexIDs, &partitionIDs); err != nil { + return errors.Trace(err) + } + indexNames = append(indexNames, indexName) } - if len(partitionIDs) > 0 { - for _, pid := range partitionIDs { - startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) - if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { - return errors.Trace(err) + + for _, indexID := range indexIDs { + if len(partitionIDs) > 0 { + for _, pid := range partitionIDs { + startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) + if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + return errors.Trace(err) + } } + } else { + startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + return doInsert(s, job.ID, indexID, startKey, endKey, now) } - } else { - startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) - return doInsert(s, job.ID, indexID, startKey, endKey, now) } case model.ActionDropColumn: var colName model.CIStr From de673ef3aa5ab0cb4ae03894d896400852f736e3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 15 Oct 2020 23:20:33 +0800 Subject: [PATCH 08/61] remove func: onDropIndex --- ddl/index.go | 75 ---------------------------------------------------- 1 file changed, 75 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 8779cf3a1503c..6e141592d8b6a 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -599,81 +599,6 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo return ver, errors.Trace(err) } -func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { - tblInfo, indexInfo, err := checkDropIndex(t, job) - if err != nil { - return ver, errors.Trace(err) - } - - dependentHiddenCols := make([]*model.ColumnInfo, 0) - for _, indexColumn := range indexInfo.Columns { - if tblInfo.Columns[indexColumn.Offset].Hidden { - dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) - } - } - - originalState := indexInfo.State - switch indexInfo.State { - case model.StatePublic: - // public -> write only - job.SchemaState = model.StateWriteOnly - indexInfo.State = model.StateWriteOnly - if len(dependentHiddenCols) > 0 { - firstHiddenOffset := dependentHiddenCols[0].Offset - for i := 0; i < len(dependentHiddenCols); i++ { - tblInfo.Columns[firstHiddenOffset].State = model.StateWriteOnly - // Set this column's offset to the last and reset all following columns' offsets. - adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) - } - } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) - case model.StateWriteOnly: - // write only -> delete only - job.SchemaState = model.StateDeleteOnly - indexInfo.State = model.StateDeleteOnly - updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) - case model.StateDeleteOnly: - // delete only -> reorganization - job.SchemaState = model.StateDeleteReorganization - indexInfo.State = model.StateDeleteReorganization - updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) - case model.StateDeleteReorganization: - // reorganization -> absent - 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) - - tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(dependentHiddenCols)] - - ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, originalState != model.StateNone) - if err != nil { - return ver, errors.Trace(err) - } - - // Finish this job. - if job.IsRollingback() { - job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) - job.Args[0] = indexInfo.ID - // the partition ids were append by convertAddIdxJob2RollbackJob, it is weird, but for the compatibility, - // we should keep appending the partitions in the convertAddIdxJob2RollbackJob. - } else { - job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - job.Args = append(job.Args, indexInfo.ID, getPartitionIDs(tblInfo)) - } - default: - err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo.State) - } - return ver, errors.Trace(err) -} - func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo, indexesInfo, err := checkDropIndexes(t, job) if err != nil { From 6fdddca6badbbcbadc5e3e075f55e93a56535862 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 16 Oct 2020 09:17:54 +0800 Subject: [PATCH 09/61] remove func: rollingbackDropIndex --- ddl/rollingback.go | 30 ------------------------------ 1 file changed, 30 deletions(-) diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 9aca28033fa21..cadc6fd0b8c16 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -260,36 +260,6 @@ func rollingbackDropColumns(t *meta.Meta, job *model.Job) (ver int64, err error) return ver, nil } -func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { - tblInfo, indexInfo, err := checkDropIndex(t, job) - if err != nil { - return ver, errors.Trace(err) - } - - originalState := indexInfo.State - switch indexInfo.State { - case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: - // 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: - job.State = model.JobStateRollbackDone - indexInfo.State = model.StatePublic - default: - return ver, ErrInvalidDDLState.GenWithStackByArgs("index", 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) - } - job.FinishTableJob(model.JobStateRollbackDone, model.StatePublic, ver, tblInfo) - return ver, errCancelledDDLJob -} - func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) { tblInfo, indexesInfo, err := checkDropIndexes(t, job) if err != nil { From acb14979ec509377690c21e9c81b57d3fb615e04 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 16 Oct 2020 09:18:02 +0800 Subject: [PATCH 10/61] remove func: checkDropIndex --- ddl/index.go | 44 -------------------------------------------- 1 file changed, 44 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 6e141592d8b6a..0878af6d38149 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -751,50 +751,6 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return tblInfo, indexesInfo, nil } -func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.IndexInfo, error) { - schemaID := job.SchemaID - tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID) - if err != nil { - return nil, nil, errors.Trace(err) - } - - var indexName model.CIStr - if err = job.DecodeArgs(&indexName); err != nil { - job.State = model.JobStateCancelled - return nil, nil, errors.Trace(err) - } - - indexInfo := tblInfo.FindIndexByName(indexName.L) - if indexInfo == nil { - job.State = model.JobStateCancelled - return nil, nil, 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 - } - - // Check that drop primary index will not cause invisible implicit primary index. - newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if idx.Name.L != indexInfo.Name.L { - newIndices = append(newIndices, idx) - } - } - newTbl := tblInfo.Clone() - newTbl.Indices = newIndices - err = checkInvisibleIndexOnPK(newTbl) - if err != nil { - job.State = model.JobStateCancelled - return nil, nil, errors.Trace(err) - } - - return tblInfo, indexInfo, nil -} - func checkDropIndexOnAutoIncrementColumn(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { cols := tblInfo.Columns for _, idxCol := range indexInfo.Columns { From 7489ad4beeaff4f6e01e97d10eaad30a82e95e25 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 16 Oct 2020 09:41:18 +0800 Subject: [PATCH 11/61] update func: insertJobIntoDeleteRangeTable --- ddl/delete_range.go | 26 ++++++++++++++------------ 1 file changed, 14 insertions(+), 12 deletions(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 9ea7dc1d631da..0d42ed0df4037 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -303,23 +303,25 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error // ActionAddIndex, ActionAddPrimaryKey needs do it, because it needs to be rolled back when it's canceled. case model.ActionAddIndex, model.ActionAddPrimaryKey: tableID := job.TableID - var indexID int64 + var indexIDs []int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexID, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexIDs, &partitionIDs); err != nil { return errors.Trace(err) } - if len(partitionIDs) > 0 { - for _, pid := range partitionIDs { - startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) - if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { - return errors.Trace(err) + for _, indexID := range indexIDs { + if len(partitionIDs) > 0 { + for _, pid := range partitionIDs { + startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) + if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + return errors.Trace(err) + } } + } else { + startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + return doInsert(s, job.ID, indexID, startKey, endKey, now) } - } else { - startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) - return doInsert(s, job.ID, indexID, startKey, endKey, now) } case model.ActionDropIndex, model.ActionDropPrimaryKey: tableID := job.TableID From 213089e9c3c8439cbcb3f8d5ce69c0c85b392da7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sat, 17 Oct 2020 19:43:22 +0800 Subject: [PATCH 12/61] modify variable type --- ddl/ddl_api.go | 2 +- ddl/delete_range.go | 1 + ddl/index.go | 1 + 3 files changed, 3 insertions(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7df572b6b66d5..947259eb731f5 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4904,7 +4904,7 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt } allIndexIsExists := true - indexNames := make([]interface{}, 0, len(specs)) + indexNames := make([]model.CIStr, 0, len(specs)) for _, spec := range specs { var indexName model.CIStr switch spec.Tp { diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 0d42ed0df4037..231bb3cf388d9 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -329,6 +329,7 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error var indexIDs []int64 var partitionIDs []int64 if err := job.DecodeArgs(&indexNames, &indexIDs, &partitionIDs); err != nil { + // Compatible with the previous function: DropIndex var indexName model.CIStr if err := job.DecodeArgs(&indexName, &indexIDs, &partitionIDs); err != nil { return errors.Trace(err) diff --git a/ddl/index.go b/ddl/index.go index 0878af6d38149..3f5921b247244 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -703,6 +703,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. var indexNames []model.CIStr if err = job.DecodeArgs(&indexNames); err != nil { + // Compatible with the previous function: DropIndex var indexName model.CIStr if err = job.DecodeArgs(&indexName); err != nil { job.State = model.JobStateCancelled From b0301ce83a06c20d3e36e74a10b048c33de9f21e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 18 Oct 2020 22:50:46 +0800 Subject: [PATCH 13/61] update func: DropIndexes --- ddl/ddl_api.go | 26 ++++++++++++++++---------- 1 file changed, 16 insertions(+), 10 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 947259eb731f5..d2745dfe5686a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4840,7 +4840,6 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI if isPK { if !config.GetGlobalConfig().AlterPrimaryKey { return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") - } // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. if indexInfo == nil && !t.Meta().PKIsHandle { @@ -4903,7 +4902,7 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) } - allIndexIsExists := true + existsSpec := make(map[string]bool, len(specs)) indexNames := make([]model.CIStr, 0, len(specs)) for _, spec := range specs { var indexName model.CIStr @@ -4953,11 +4952,20 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt return errors.Trace(err) } + if _, ok := existsSpec[indexName.L]; ok { + return ErrCantDropFieldOrKey.GenWithStack("Can't DROP '%s'; check that column/key exists", indexName) + } + if spec.IfExists { + existsSpec[indexName.L] = true + } else { + existsSpec[indexName.L] = false + } indexNames = append(indexNames, indexName) + } - if !spec.IfExists { - allIndexIsExists = false - } + ifExists := make([]bool, 0, len(indexNames)) + for _, v := range existsSpec { + ifExists = append(ifExists, v) } job := &model.Job{ @@ -4966,14 +4974,12 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt SchemaName: schema.Name.L, Type: model.ActionDropIndex, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{indexNames}, + Args: []interface{}{indexNames, ifExists}, } err = d.doDDLJob(ctx, job) - // index not exists, but if_exists flags is true, so we ignore this error. - if ErrCantDropFieldOrKey.Equal(err) && allIndexIsExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil + if err != nil { + return errors.Trace(err) } err = d.callHookOnChanged(err) return errors.Trace(err) From 2035b2871b3072b82780467e67f25a08c95a0bba Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 18 Oct 2020 22:51:15 +0800 Subject: [PATCH 14/61] update func: insertJobIntoDeleteRangeTable --- ddl/delete_range.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 231bb3cf388d9..9b8f52460a131 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -326,9 +326,10 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error case model.ActionDropIndex, model.ActionDropPrimaryKey: tableID := job.TableID var indexNames []model.CIStr + var ifExists []bool var indexIDs []int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexNames, &indexIDs, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexNames, &ifExists, &indexIDs, &partitionIDs); err != nil { // Compatible with the previous function: DropIndex var indexName model.CIStr if err := job.DecodeArgs(&indexName, &indexIDs, &partitionIDs); err != nil { From 6211065fb955c87da7bfa12560814dede66fc061 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 18 Oct 2020 22:51:30 +0800 Subject: [PATCH 15/61] update func: checkDropIndexes --- ddl/index.go | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 3f5921b247244..33759b351f713 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -702,7 +702,8 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. } var indexNames []model.CIStr - if err = job.DecodeArgs(&indexNames); err != nil { + var ifExists []bool + if err = job.DecodeArgs(&indexNames, &ifExists); err != nil { // Compatible with the previous function: DropIndex var indexName model.CIStr if err = job.DecodeArgs(&indexName); err != nil { @@ -710,13 +711,14 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return nil, nil, errors.Trace(err) } indexNames = append(indexNames, indexName) + ifExists = append(ifExists, false) } indexesInfo := make([]*model.IndexInfo, 0, len(indexNames)) indexesMap := make(map[string]bool) - for _, indexName := range indexNames { + for i, indexName := range indexNames { indexInfo := tblInfo.FindIndexByName(indexName.L) - if indexInfo == nil { + if indexInfo == nil && !ifExists[i] { job.State = model.JobStateCancelled return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } From 716fa7937f5285f3384a288fa10d09f06a0686c0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 21 Oct 2020 16:18:08 +0800 Subject: [PATCH 16/61] update func: insertJobIntoDeleteRangeTable --- ddl/delete_range.go | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 9b8f52460a131..ff569fe890989 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -324,7 +324,6 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error } } case model.ActionDropIndex, model.ActionDropPrimaryKey: - tableID := job.TableID var indexNames []model.CIStr var ifExists []bool var indexIDs []int64 @@ -338,19 +337,15 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error indexNames = append(indexNames, indexName) } - for _, indexID := range indexIDs { + if len(indexIDs) > 0 { if len(partitionIDs) > 0 { for _, pid := range partitionIDs { - startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) - if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { return errors.Trace(err) } } } else { - startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) - return doInsert(s, job.ID, indexID, startKey, endKey, now) + return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) } } case model.ActionDropColumn: From 68032f41a214623d6bad3878c3b76c2bf83eeed8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 21 Oct 2020 16:18:18 +0800 Subject: [PATCH 17/61] add test case: TestDropIndexes --- ddl/db_test.go | 78 ++++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 78 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index 1e1848b44fcba..5e28f80432889 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1480,6 +1480,84 @@ LOOP: tk.MustExec("drop table test_drop_index") } +func (s *testDBSuite2) TestDropIndexes(c *C) { + idxNames := []string{"c2_index", "c3_index"} + createSQL := "create table test_drop_indexes (c1 int, c2 int, c3 int, unique key(c1), key c2_index(c2), key c3_index(c3))" + dropIdxSQL := "alter table test_drop_indexes drop index c2_index, drop index c3_index;" + testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) +} + +func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dropIdxSQL string, idxNames []string) { + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists test_drop_indexes") + tk.MustExec(createSQL) + done := make(chan error, 1) + tk.MustExec("delete from test_drop_indexes") + + num := 100 + // add some rows + for i := 0; i < num; i++ { + tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) + } + ctx := tk.Se.(sessionctx.Context) + t := testGetTableByName(c, ctx, "test_db", "test_drop_indexes") + var idxs []table.Index + for _, tidx := range t.Indices() { + for _, idxName := range idxNames { + if tidx.Meta().Name.L == idxName { + idxs = append(idxs, tidx) + break + } + } + } + c.Assert(idxs, NotNil) + + testddlutil.SessionExecInGoroutine(c, store, dropIdxSQL, done) + + ticker := time.NewTicker(lease / 2) + defer ticker.Stop() +LOOP: + for { + select { + case err := <-done: + if err == nil { + break LOOP + } + c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) + case <-ticker.C: + step := 5 + // delete some rows, and add some data + for i := num; i < num+step; i++ { + n := rand.Intn(num) + tk.MustExec("update test_drop_indexes set c2 = 1 where c1 = ?", n) + tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) + } + num += step + } + } + + // Check in index, it must be no index in KV. + // Make sure there is no index with name c2_index、c3_index. + t = testGetTableByName(c, ctx, "test_db", "test_drop_indexes") + var nidxs []table.Index + for _, tidx := range t.Indices() { + for _, ids := range idxs { + if tidx.Meta().Name.L == ids.Meta().Name.L { + nidxs = append(nidxs, tidx) + } + } + } + c.Assert(nidxs, IsNil) + + for _, idx := range idxs { + idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) + checkDelRangeDone(c, ctx, idx) + } + + tk.MustExec("drop table test_drop_indexes") +} + // TestCancelDropColumn tests cancel ddl job which type is drop column. func (s *testDBSuite3) TestCancelDropColumn(c *C) { tk := testkit.NewTestKit(c, s.store) From ff0d5d778aad9f0628217e031526ac531ec1e6ad Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 21 Oct 2020 17:07:25 +0800 Subject: [PATCH 18/61] update func: onDropIndexes --- ddl/index.go | 27 +++++++++------------------ 1 file changed, 9 insertions(+), 18 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 33759b351f713..473487008b2be 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -618,15 +618,12 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } - indexInfo := indexesInfo[0] - originalState := indexInfo.State - switch indexInfo.State { + originalState := indexesInfo[0].State + switch indexesInfo[0].State { case model.StatePublic: // public -> write only job.SchemaState = model.StateWriteOnly - for _, indexInfo := range indexesInfo { - indexInfo.State = model.StateWriteOnly - } + setIndicesState(indexesInfo, model.StateWriteOnly) if len(dependentHiddenCols) > 0 { firstHiddenOffset := dependentHiddenCols[0].Offset for i := 0; i < len(dependentHiddenCols); i++ { @@ -635,23 +632,17 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) } } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) case model.StateWriteOnly: // write only -> delete only job.SchemaState = model.StateDeleteOnly - for _, indexInfo := range indexesInfo { - indexInfo.State = model.StateDeleteOnly - updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) - } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + setIndicesState(indexesInfo, model.StateDeleteOnly) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) case model.StateDeleteOnly: // delete only -> reorganization job.SchemaState = model.StateDeleteReorganization - for _, indexInfo := range indexesInfo { - indexInfo.State = model.StateDeleteReorganization - updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) - } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + setIndicesState(indexesInfo, model.StateDeleteReorganization) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) case model.StateDeleteReorganization: indexesIDs := make([]int64, 0, len(indexesInfo)) indexesLName := make(map[string]bool, len(indexesInfo)) @@ -689,7 +680,7 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.Args = append(job.Args, indexesIDs, getPartitionIDs(tblInfo)) } default: - err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo.State) + err = ErrInvalidDDLState.GenWithStackByArgs("index", indexesInfo[0].State) } return ver, errors.Trace(err) } From 7eb7b13a9eda3dcde7e684d9565dd59d337e8efb Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 22 Oct 2020 23:45:36 +0800 Subject: [PATCH 19/61] =?UTF-8?q?refactor=20functions:=20DropIndex?= =?UTF-8?q?=E3=80=81DropIndexes?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit --- ddl/ddl_api.go | 127 +++++++++++++++++++------------------------------ 1 file changed, 48 insertions(+), 79 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index d2745dfe5686a..6f991ea4ee5db 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4819,19 +4819,22 @@ func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model. return errors.Trace(err) } -func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, ifExists bool) error { +func (d *ddl) getSchemaAndTable(ti ast.Ident) (*model.DBInfo, table.Table, error) { is := d.infoHandle.Get() schema, ok := is.SchemaByName(ti.Schema) if !ok { - return errors.Trace(infoschema.ErrDatabaseNotExists) + return nil, nil, errors.Trace(infoschema.ErrDatabaseNotExists) } t, err := is.TableByName(ti.Schema, ti.Name) if err != nil { - return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + return nil, nil, errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) } + return schema, t, nil +} +func checkIndexInfo(ctx sessionctx.Context, t table.Table, indexName model.CIStr, ifExists bool) (isPK bool, shouldIgnore bool, _ error) { indexInfo := t.Meta().FindIndexByName(indexName.L) - var isPK bool + if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && // Before we fixed #14243, there might be a general index named `primary` but not a primary key. (indexInfo == nil || indexInfo.Primary) { @@ -4839,32 +4842,48 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI } if isPK { if !config.GetGlobalConfig().AlterPrimaryKey { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") + return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") } // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. if indexInfo == nil && !t.Meta().PKIsHandle { - return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") + return isPK, shouldIgnore, ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") } if t.Meta().PKIsHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") + return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") } if t.Meta().IsCommonHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") } } + if indexInfo == nil { - err = ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) if ifExists { ctx.GetSessionVars().StmtCtx.AppendNote(err) - return nil + return isPK, true, nil } - return err + return isPK, shouldIgnore, err } // Check for drop index on auto_increment column. - err = checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo) + if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { + return isPK, shouldIgnore, errors.Trace(err) + } + return isPK, shouldIgnore, nil +} + +func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, ifExists bool) error { + schema, t, err := d.getSchemaAndTable(ti) if err != nil { - return errors.Trace(err) + return err + } + + isPK, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, ifExists) + if err != nil { + return err + } + if shouldIgnore { + return nil } jobTp := model.ActionDropIndex @@ -4892,80 +4911,30 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI } func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.AlterTableSpec) error { - is := d.infoHandle.Get() - schema, ok := is.SchemaByName(ti.Schema) - if !ok { - return errors.Trace(infoschema.ErrDatabaseNotExists) - } - t, err := is.TableByName(ti.Schema, ti.Name) + schema, t, err := d.getSchemaAndTable(ti) if err != nil { - return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + return err } - existsSpec := make(map[string]bool, len(specs)) - indexNames := make([]model.CIStr, 0, len(specs)) + indexesNames := make([]model.CIStr, 0, len(specs)) + ifExists := make([]bool, 0, len(specs)) for _, spec := range specs { - var indexName model.CIStr - switch spec.Tp { - case ast.AlterTableDropIndex: - indexName = model.NewCIStr(spec.Name) - case ast.AlterTableDropPrimaryKey: - indexName = model.NewCIStr(mysql.PrimaryKeyName) - default: - return nil - } - - indexInfo := t.Meta().FindIndexByName(indexName.L) - var isPK bool - if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && - // Before we fixed #14243, there might be a general index named `primary` but not a primary key. - (indexInfo == nil || indexInfo.Primary) { - isPK = true - } - if isPK { - if !config.GetGlobalConfig().AlterPrimaryKey { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") - } - // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. - if indexInfo == nil && !t.Meta().PKIsHandle { - return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") - } - if t.Meta().PKIsHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") - } - if t.Meta().IsCommonHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") - } - } - if indexInfo == nil { - err = ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) - if spec.IfExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - continue - } + indexName := model.NewCIStr(spec.Name) + _, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, spec.IfExists) + if err != nil { return err } - - // Check for drop index on auto_increment column. - err = checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo) - if err != nil { - return errors.Trace(err) + if shouldIgnore { + continue } - if _, ok := existsSpec[indexName.L]; ok { - return ErrCantDropFieldOrKey.GenWithStack("Can't DROP '%s'; check that column/key exists", indexName) - } - if spec.IfExists { - existsSpec[indexName.L] = true - } else { - existsSpec[indexName.L] = false + for _, idxName := range indexesNames { + if idxName == indexName { + return ErrCantDropFieldOrKey.GenWithStack("Can't DROP '%s'; check that column/key exists", indexName) + } } - indexNames = append(indexNames, indexName) - } - - ifExists := make([]bool, 0, len(indexNames)) - for _, v := range existsSpec { - ifExists = append(ifExists, v) + indexesNames = append(indexesNames, indexName) + ifExists = append(ifExists, spec.IfExists) } job := &model.Job{ @@ -4974,7 +4943,7 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt SchemaName: schema.Name.L, Type: model.ActionDropIndex, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{indexNames, ifExists}, + Args: []interface{}{indexesNames, ifExists}, } err = d.doDDLJob(ctx, job) From 28bf03963de8cb52250785e17a0b6537f8cd3b2c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 23 Oct 2020 10:12:23 +0800 Subject: [PATCH 20/61] refactor func: checkDropIndexes --- ddl/index.go | 35 ++++++++++++++++++----------------- 1 file changed, 18 insertions(+), 17 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 473487008b2be..d7be283098fb7 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -604,10 +604,6 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { if err != nil { return ver, errors.Trace(err) } - if len(indexesInfo) == 0 { - job.State = model.JobStateCancelled - return ver, nil - } dependentHiddenCols := make([]*model.ColumnInfo, 0) for _, indexInfo := range indexesInfo { @@ -618,6 +614,10 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } + if len(indexesInfo) == 0 { + job.State = model.JobStateCancelled + return ver, nil + } originalState := indexesInfo[0].State switch indexesInfo[0].State { case model.StatePublic: @@ -692,26 +692,29 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return nil, nil, errors.Trace(err) } - var indexNames []model.CIStr + var indexesNames []model.CIStr var ifExists []bool - if err = job.DecodeArgs(&indexNames, &ifExists); err != nil { + if err = job.DecodeArgs(&indexesNames, &ifExists); err != nil { // Compatible with the previous function: DropIndex var indexName model.CIStr if err = job.DecodeArgs(&indexName); err != nil { job.State = model.JobStateCancelled return nil, nil, errors.Trace(err) } - indexNames = append(indexNames, indexName) + indexesNames = append(indexesNames, indexName) ifExists = append(ifExists, false) } - indexesInfo := make([]*model.IndexInfo, 0, len(indexNames)) - indexesMap := make(map[string]bool) - for i, indexName := range indexNames { + indexesInfo := make([]*model.IndexInfo, 0, len(indexesNames)) + droppedIndexes := make(map[string]bool, len(indexesNames)) + for i, indexName := range indexesNames { indexInfo := tblInfo.FindIndexByName(indexName.L) - if indexInfo == nil && !ifExists[i] { - job.State = model.JobStateCancelled - return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + if indexInfo == nil { + if !ifExists[i] { + job.State = model.JobStateCancelled + return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + } + continue } // Double check for drop index on auto_increment column. @@ -722,18 +725,16 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. } indexesInfo = append(indexesInfo, indexInfo) - indexesMap[indexName.L] = true + droppedIndexes[indexName.L] = true } // Check that drop primary index will not cause invisible implicit primary index. newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if _, ok := indexesMap[idx.Name.L]; !ok { + if _, ok := droppedIndexes[idx.Name.L]; !ok { newIndices = append(newIndices, idx) } } - newTbl := tblInfo.Clone() newTbl.Indices = newIndices err = checkInvisibleIndexOnPK(newTbl) From d4cc82ec9a0ba741bdee69fa2997521619e25fc1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 25 Oct 2020 15:13:26 +0800 Subject: [PATCH 21/61] add conditional branch --- ddl/ddl_api.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 6f991ea4ee5db..2112178444864 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4937,6 +4937,10 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt ifExists = append(ifExists, spec.IfExists) } + if len(indexesNames) == 0 { + return nil + } + job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, From ec2686876530adcd85277dd1c7d92746f429bc2b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 25 Oct 2020 23:24:34 +0800 Subject: [PATCH 22/61] add updateHiddenColumns --- ddl/index.go | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index d7be283098fb7..734dfe98d6928 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -605,6 +605,11 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } + if len(indexesInfo) == 0 { + job.State = model.JobStateCancelled + return ver, nil + } + dependentHiddenCols := make([]*model.ColumnInfo, 0) for _, indexInfo := range indexesInfo { for _, indexColumn := range indexInfo.Columns { @@ -614,10 +619,6 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } - if len(indexesInfo) == 0 { - job.State = model.JobStateCancelled - return ver, nil - } originalState := indexesInfo[0].State switch indexesInfo[0].State { case model.StatePublic: @@ -636,20 +637,28 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { case model.StateWriteOnly: // write only -> delete only job.SchemaState = model.StateDeleteOnly - setIndicesState(indexesInfo, model.StateDeleteOnly) + for _, indexInfo := range indexesInfo { + indexInfo.State = model.StateDeleteOnly + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) + } ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) case model.StateDeleteOnly: // delete only -> reorganization job.SchemaState = model.StateDeleteReorganization - setIndicesState(indexesInfo, model.StateDeleteReorganization) + for _, indexInfo := range indexesInfo { + indexInfo.State = model.StateDeleteReorganization + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) + } ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) case model.StateDeleteReorganization: + // reorganization -> absent indexesIDs := make([]int64, 0, len(indexesInfo)) indexesLName := make(map[string]bool, len(indexesInfo)) for _, indexInfo := range indexesInfo { indexesLName[indexInfo.Name.L] = true indexesIDs = append(indexesIDs, indexInfo.ID) } + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) for _, idx := range tblInfo.Indices { if _, ok := indexesLName[idx.Name.L]; !ok { @@ -657,6 +666,7 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } tblInfo.Indices = newIndices + // Set column index flag. for _, indexInfo := range indexesInfo { dropIndexColumnFlag(tblInfo, indexInfo) From 1aef3d597fca1c471843ed537e7a9b85b371b38f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 26 Oct 2020 16:10:36 +0800 Subject: [PATCH 23/61] add test case --- ddl/db_test.go | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index 5e28f80432889..d385c22c8bd89 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -4311,6 +4311,10 @@ func (s *testDBSuite4) TestIfExists(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index idx_c doesn't exist")) + // DROP INDEXES + s.mustExec(tk, c, "alter table t1 drop index if exists idxes_a,drop index if exists idxes_b") + c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) + // DROP PARTITION s.mustExec(tk, c, "drop table if exists t2") s.mustExec(tk, c, "create table t2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") From 7355982b66618575e16f28075356aea8c5a44e51 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 26 Oct 2020 17:09:41 +0800 Subject: [PATCH 24/61] add test case --- ddl/db_test.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index d385c22c8bd89..47a5f53c538b1 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1482,9 +1482,15 @@ LOOP: func (s *testDBSuite2) TestDropIndexes(c *C) { idxNames := []string{"c2_index", "c3_index"} - createSQL := "create table test_drop_indexes (c1 int, c2 int, c3 int, unique key(c1), key c2_index(c2), key c3_index(c3))" + createSQL := "create table test_drop_indexes (c1 int, c2 int, c3 int, primary key(c1), key c2_index(c2), key c3_index(c3))" dropIdxSQL := "alter table test_drop_indexes drop index c2_index, drop index c3_index;" testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) + + // test drop primary key and index + idxNames = []string{"primary", "c2_index"} + createSQL = "create table test_drop_indexes (c1 int, c2 int, c3 int, primary key(c1), key c2_index(c2), key c3_index(c3))" + dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index c2_index;" + testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) } func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dropIdxSQL string, idxNames []string) { From a72172a258898b5537c60369f1b2d83a6e54d961 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 26 Oct 2020 17:10:17 +0800 Subject: [PATCH 25/61] fix drop primary key and index --- ddl/ddl_api.go | 26 +++++++++++++++++++++++--- 1 file changed, 23 insertions(+), 3 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2112178444864..ae6e19cc21cae 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2260,6 +2260,15 @@ func isSameTypeMultiSpecs(specs []*ast.AlterTableSpec) bool { return true } +func isDropIndexes(specs []*ast.AlterTableSpec) bool { + for _, spec := range specs { + if spec.Tp != ast.AlterTableDropPrimaryKey && spec.Tp != ast.AlterTableDropIndex { + return false + } + } + return true +} + func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) { validSpecs, err := resolveAlterTableSpec(ctx, specs) if err != nil { @@ -2272,14 +2281,19 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } if len(validSpecs) > 1 { + if isDropIndexes(validSpecs) { + if err = d.DropIndexes(ctx, ident, validSpecs); err != nil { + return errors.Trace(err) + } + return nil + } + if isSameTypeMultiSpecs(validSpecs) { switch validSpecs[0].Tp { case ast.AlterTableAddColumns: err = d.AddColumns(ctx, ident, validSpecs) case ast.AlterTableDropColumn: err = d.DropColumns(ctx, ident, validSpecs) - case ast.AlterTableDropIndex: - err = d.DropIndexes(ctx, ident, validSpecs) default: return errRunMultiSchemaChanges } @@ -2288,6 +2302,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } return nil } + return errRunMultiSchemaChanges } @@ -4919,7 +4934,12 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt indexesNames := make([]model.CIStr, 0, len(specs)) ifExists := make([]bool, 0, len(specs)) for _, spec := range specs { - indexName := model.NewCIStr(spec.Name) + var indexName model.CIStr + if spec.Tp == ast.AlterTableDropPrimaryKey { + indexName = model.NewCIStr(mysql.PrimaryKeyName) + } else { + indexName = model.NewCIStr(spec.Name) + } _, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, spec.IfExists) if err != nil { return err From 5cd8b495a518fc03675dbe2172f5b9f1e505c335 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 26 Oct 2020 22:28:37 +0800 Subject: [PATCH 26/61] add test case --- ddl/db_test.go | 105 +++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 105 insertions(+) diff --git a/ddl/db_test.go b/ddl/db_test.go index 47a5f53c538b1..58248771782a4 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -607,6 +607,111 @@ func (s *testDBSuite4) TestCancelDropPrimaryKey(c *C) { testCancelDropIndex(c, s.store, s.dom.DDL(), idxName, addIdxSQL, dropIdxSQL) } +func (s *testDBSuite5) TestCancelDropIndexes(c *C) { + indexesName := []string{"idx_c1", "idx_c2"} + addIdxesSQL := "alter table t add index idx_c1 (c1);alter table t add index idx_c2 (c2);" + dropIdxesSQL := "alter table t drop index idx_c1;alter table t drop index idx_c2;" + + store := s.store + d := s.dom.DDL() + + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int, c2 int)") + defer tk.MustExec("drop table t;") + for i := 0; i < 5; i++ { + tk.MustExec("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. + // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. + {true, model.JobStateNone, model.StateNone, true}, + {false, model.JobStateRunning, model.StateWriteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteReorganization, false}, + } + var checkErr error + hook := &ddl.TestDDLCallback{} + var jobID int64 + testCase := &testCases[0] + hook.OnJobRunBeforeExported = func(job *model.Job) { + if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && + job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { + jobID = job.ID + jobIDs := []int64{job.ID} + hookCtx := mock.NewContext() + hookCtx.Store = store + err := hookCtx.NewTxn(context.TODO()) + if err != nil { + checkErr = errors.Trace(err) + return + } + txn, err := hookCtx.Txn(true) + if err != nil { + checkErr = errors.Trace(err) + return + } + + errs, err := admin.CancelJobs(txn, jobIDs) + if err != nil { + checkErr = errors.Trace(err) + return + } + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + checkErr = txn.Commit(context.Background()) + } + } + originalHook := d.GetHook() + d.(ddl.DDLForTest).SetHook(hook) + ctx := tk.Se.(sessionctx.Context) + for i := range testCases { + testCase = &testCases[i] + if testCase.needAddIndex { + tk.MustExec(addIdxesSQL) + } + rs, err := tk.Exec(dropIdxesSQL) + if rs != nil { + rs.Close() + } + t := testGetTableByName(c, ctx, "test_db", "t") + + var indexesInfo []*model.IndexInfo + for _, idxName := range indexesName { + indexInfo := t.Meta().FindIndexByName(idxName) + if indexInfo != nil { + indexesInfo = append(indexesInfo, indexInfo) + } + } + + if testCase.cancelSucc { + c.Assert(checkErr, IsNil) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") + c.Assert(indexesInfo, NotNil) + c.Assert(indexesInfo[0].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(indexesInfo, IsNil) + } + } + d.(ddl.DDLForTest).SetHook(originalHook) + tk.MustExec(addIdxesSQL) + tk.MustExec(dropIdxesSQL) + +} + // TestCancelDropIndex tests cancel ddl job which type is drop index. func (s *testDBSuite5) TestCancelDropIndex(c *C) { idxName := "idx_c2" From fcd5a23d3af0723b1b92182ab7d48ae05de564e6 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 30 Oct 2020 14:31:00 +0800 Subject: [PATCH 27/61] Optimize the code --- ddl/index.go | 51 +++++++++++++++++++++------------------------------ 1 file changed, 21 insertions(+), 30 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 734dfe98d6928..cc77e15a9e51a 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -600,18 +600,17 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo } func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { - tblInfo, indexesInfo, err := checkDropIndexes(t, job) + tblInfo, idxInfos, err := checkDropIndexes(t, job) if err != nil { return ver, errors.Trace(err) } - - if len(indexesInfo) == 0 { + if len(idxInfos) == 0 { job.State = model.JobStateCancelled return ver, nil } dependentHiddenCols := make([]*model.ColumnInfo, 0) - for _, indexInfo := range indexesInfo { + for _, indexInfo := range idxInfos { for _, indexColumn := range indexInfo.Columns { if tblInfo.Columns[indexColumn.Offset].Hidden { dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) @@ -619,42 +618,34 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } - originalState := indexesInfo[0].State - switch indexesInfo[0].State { + originalState := idxInfos[0].State + switch idxInfos[0].State { case model.StatePublic: // public -> write only job.SchemaState = model.StateWriteOnly - setIndicesState(indexesInfo, model.StateWriteOnly) - if len(dependentHiddenCols) > 0 { - firstHiddenOffset := dependentHiddenCols[0].Offset - for i := 0; i < len(dependentHiddenCols); i++ { - tblInfo.Columns[firstHiddenOffset].State = model.StateWriteOnly - // Set this column's offset to the last and reset all following columns' offsets. - adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) - } + setIndicesState(idxInfos, model.StateWriteOnly) + setColumnsState(dependentHiddenCols, model.StateWriteOnly) + for _, colInfo := range dependentHiddenCols { + adjustColumnInfoInDropColumn(tblInfo, colInfo.Offset) } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) case model.StateWriteOnly: // write only -> delete only job.SchemaState = model.StateDeleteOnly - for _, indexInfo := range indexesInfo { - indexInfo.State = model.StateDeleteOnly - updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) - } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) + setIndicesState(idxInfos, model.StateDeleteOnly) + setColumnsState(dependentHiddenCols, model.StateDeleteOnly) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) case model.StateDeleteOnly: // delete only -> reorganization job.SchemaState = model.StateDeleteReorganization - for _, indexInfo := range indexesInfo { - indexInfo.State = model.StateDeleteReorganization - updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) - } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexesInfo[0].State) + setIndicesState(idxInfos, model.StateDeleteReorganization) + setColumnsState(dependentHiddenCols, model.StateDeleteReorganization) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) case model.StateDeleteReorganization: // reorganization -> absent - indexesIDs := make([]int64, 0, len(indexesInfo)) - indexesLName := make(map[string]bool, len(indexesInfo)) - for _, indexInfo := range indexesInfo { + indexesIDs := make([]int64, 0, len(idxInfos)) + indexesLName := make(map[string]bool, len(idxInfos)) + for _, indexInfo := range idxInfos { indexesLName[indexInfo.Name.L] = true indexesIDs = append(indexesIDs, indexInfo.ID) } @@ -668,7 +659,7 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo.Indices = newIndices // Set column index flag. - for _, indexInfo := range indexesInfo { + for _, indexInfo := range idxInfos { dropIndexColumnFlag(tblInfo, indexInfo) } @@ -690,7 +681,7 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.Args = append(job.Args, indexesIDs, getPartitionIDs(tblInfo)) } default: - err = ErrInvalidDDLState.GenWithStackByArgs("index", indexesInfo[0].State) + err = ErrInvalidDDLState.GenWithStackByArgs("index", idxInfos[0].State) } return ver, errors.Trace(err) } From 9fbd6257a099ce522ae8ae221fd216bd2d4a9745 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 30 Oct 2020 16:52:39 +0800 Subject: [PATCH 28/61] Optimize the code --- ddl/ddl_api.go | 38 ++++++++++++++++++++++++-------------- ddl/index.go | 17 +++++++---------- 2 files changed, 31 insertions(+), 24 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index ae6e19cc21cae..21660ced44324 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4931,8 +4931,9 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt return err } - indexesNames := make([]model.CIStr, 0, len(specs)) - ifExists := make([]bool, 0, len(specs)) + // We need to confirm whether we should return an error when dropping duplicate indexes + indexesIfExists := make(map[string]bool, len(specs)) + droppingIndex := make([]model.CIStr, 0, len(specs)) for _, spec := range specs { var indexName model.CIStr if spec.Tp == ast.AlterTableDropPrimaryKey { @@ -4940,24 +4941,33 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt } else { indexName = model.NewCIStr(spec.Name) } - _, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, spec.IfExists) + _, ok := indexesIfExists[indexName.L] + if !ok { + indexesIfExists[indexName.L] = spec.IfExists + droppingIndex = append(droppingIndex, indexName) + } else { + if spec.IfExists { + continue + } + return ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + } + } + + // + indexNames := make([]model.CIStr, 0, len(droppingIndex)) + ifExists := make([]bool, 0, len(droppingIndex)) + for _, indexName := range droppingIndex { + _, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, indexesIfExists[indexName.L]) if err != nil { return err } if shouldIgnore { continue } - - for _, idxName := range indexesNames { - if idxName == indexName { - return ErrCantDropFieldOrKey.GenWithStack("Can't DROP '%s'; check that column/key exists", indexName) - } - } - indexesNames = append(indexesNames, indexName) - ifExists = append(ifExists, spec.IfExists) + indexNames = append(indexNames, indexName) + ifExists = append(ifExists, indexesIfExists[indexName.L]) } - - if len(indexesNames) == 0 { + if len(indexNames) == 0 { return nil } @@ -4967,7 +4977,7 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt SchemaName: schema.Name.L, Type: model.ActionDropIndex, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{indexesNames, ifExists}, + Args: []interface{}{indexNames, ifExists}, } err = d.doDDLJob(ctx, job) diff --git a/ddl/index.go b/ddl/index.go index cc77e15a9e51a..ec35d794dcadf 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -693,22 +693,23 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return nil, nil, errors.Trace(err) } - var indexesNames []model.CIStr + var indexNames []model.CIStr var ifExists []bool - if err = job.DecodeArgs(&indexesNames, &ifExists); err != nil { + if err = job.DecodeArgs(&indexNames, &ifExists); err != nil { // Compatible with the previous function: DropIndex var indexName model.CIStr if err = job.DecodeArgs(&indexName); err != nil { job.State = model.JobStateCancelled return nil, nil, errors.Trace(err) } - indexesNames = append(indexesNames, indexName) + indexNames = append(indexNames, indexName) ifExists = append(ifExists, false) } - indexesInfo := make([]*model.IndexInfo, 0, len(indexesNames)) - droppedIndexes := make(map[string]bool, len(indexesNames)) - for i, indexName := range indexesNames { + indexesInfo := make([]*model.IndexInfo, 0, len(indexNames)) + droppedIndexes := make(map[string]bool, len(indexNames)) + for i, indexName := range indexNames { + // Double check the index is exists indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { if !ifExists[i] { @@ -717,18 +718,15 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. } continue } - // 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 } - indexesInfo = append(indexesInfo, indexInfo) droppedIndexes[indexName.L] = true } - // Check that drop primary index will not cause invisible implicit primary index. newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) for _, idx := range tblInfo.Indices { @@ -743,7 +741,6 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. job.State = model.JobStateCancelled return nil, nil, errors.Trace(err) } - return tblInfo, indexesInfo, nil } From 385b5575f280d9aebd88166349c9fc26caee4f4e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 30 Oct 2020 17:33:47 +0800 Subject: [PATCH 29/61] Optimize the code --- ddl/ddl_api.go | 2 +- ddl/delete_range.go | 20 ++++++++++---------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 21660ced44324..e2d4c802e7611 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -4953,7 +4953,7 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt } } - // + // Check the index is droppable. indexNames := make([]model.CIStr, 0, len(droppingIndex)) ifExists := make([]bool, 0, len(droppingIndex)) for _, indexName := range droppingIndex { diff --git a/ddl/delete_range.go b/ddl/delete_range.go index ff569fe890989..5f076ed1fa0d1 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -336,16 +336,16 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error } indexNames = append(indexNames, indexName) } - - if len(indexIDs) > 0 { - if len(partitionIDs) > 0 { - for _, pid := range partitionIDs { - if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { - return errors.Trace(err) - } - } - } else { - return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) + // Remove data in TiKV. + if len(indexIDs) == 0 { + return nil + } + if len(partitionIDs) == 0 { + return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) + } + for _, pid := range partitionIDs { + if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { + return errors.Trace(err) } } case model.ActionDropColumn: From c68d7fb5da78c50d88272f503011db9e314cf3a2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Thu, 12 Nov 2020 21:02:22 +0800 Subject: [PATCH 30/61] fix bug for drop indexes --- ddl/db_test.go | 33 ++++++++++++++++++++++++++------- ddl/ddl_api.go | 11 ++++++----- 2 files changed, 32 insertions(+), 12 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index c91d2062c73d9..fcfbe4eb89c82 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -1586,16 +1586,35 @@ LOOP: tk.MustExec("drop table test_drop_index") } +func (s *testDBSuite2) TestDropDuplicateIndexes(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists test_drop_duplicate_indexes;") + tk.MustExec("create table test_drop_duplicate_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") + + if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index i1, drop index i1;"); true { + c.Assert(err.Error(), Equals, "[ddl:1091]index i1 doesn't exist") + } + + tk.MustExec("alter table test_drop_duplicate_indexes drop index i1, drop index if exists i1;") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index i1 doesn't exist")) + + if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index if exists i2, drop index i2;"); true { + c.Assert(err.Error(), Equals, "[ddl:1091]index i2 doesn't exist") + } + + tk.MustExec("drop table if exists test_drop_duplicate_indexes") +} + func (s *testDBSuite2) TestDropIndexes(c *C) { - idxNames := []string{"c2_index", "c3_index"} - createSQL := "create table test_drop_indexes (c1 int, c2 int, c3 int, primary key(c1), key c2_index(c2), key c3_index(c3))" - dropIdxSQL := "alter table test_drop_indexes drop index c2_index, drop index c3_index;" + // drop multiple indexes + createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2))" + dropIdxSQL := "alter table test_drop_indexes drop index i1, drop index i2;" + idxNames := []string{"i1", "i2"} testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) - // test drop primary key and index - idxNames = []string{"primary", "c2_index"} - createSQL = "create table test_drop_indexes (c1 int, c2 int, c3 int, primary key(c1), key c2_index(c2), key c3_index(c3))" - dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index c2_index;" + dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" + idxNames = []string{"primary", "i1"} testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2292427d349e1..3c6c6f18f50fd 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5235,15 +5235,16 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt indexName = model.NewCIStr(spec.Name) } _, ok := indexesIfExists[indexName.L] - if !ok { - indexesIfExists[indexName.L] = spec.IfExists - droppingIndex = append(droppingIndex, indexName) - } else { + if ok { + err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) if spec.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) continue } - return ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + return err } + indexesIfExists[indexName.L] = spec.IfExists + droppingIndex = append(droppingIndex, indexName) } // Check the index is droppable. From e56e4893652fa0783849cb3cdccb862c8758c9db Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 13 Nov 2020 14:11:08 +0800 Subject: [PATCH 31/61] Update logic for modifying job status --- ddl/index.go | 15 ++++++++++++--- 1 file changed, 12 insertions(+), 3 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 88d39eaf697b1..b953d2feb3c04 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -625,25 +625,34 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { switch idxInfos[0].State { case model.StatePublic: // public -> write only - job.SchemaState = model.StateWriteOnly setIndicesState(idxInfos, model.StateWriteOnly) setColumnsState(dependentHiddenCols, model.StateWriteOnly) for _, colInfo := range dependentHiddenCols { adjustColumnInfoInDropColumn(tblInfo, colInfo.Offset) } ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateWriteOnly case model.StateWriteOnly: // write only -> delete only - job.SchemaState = model.StateDeleteOnly setIndicesState(idxInfos, model.StateDeleteOnly) setColumnsState(dependentHiddenCols, model.StateDeleteOnly) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateDeleteOnly case model.StateDeleteOnly: // delete only -> reorganization - job.SchemaState = model.StateDeleteReorganization setIndicesState(idxInfos, model.StateDeleteReorganization) setColumnsState(dependentHiddenCols, model.StateDeleteReorganization) ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateDeleteReorganization case model.StateDeleteReorganization: // reorganization -> absent indexesIDs := make([]int64, 0, len(idxInfos)) From cca227fe4a65c547dcea96d5355f640848c87bb7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 24 Feb 2021 22:09:11 +0800 Subject: [PATCH 32/61] typo --- ddl/ddl_api.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 5d7790a965cb9..926c2de8fbbf6 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2337,17 +2337,17 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } if len(validSpecs) > 1 { - if !ctx.GetSessionVars().EnableChangeMultiSchema { + if !ctx.GetSessionVars().EnableChangeMultiSchema { return errRunMultiSchemaChanges } - + if isDropIndexes(validSpecs) { if err = d.DropIndexes(ctx, ident, validSpecs); err != nil { return errors.Trace(err) } return nil } - + if isSameTypeMultiSpecs(validSpecs) { switch validSpecs[0].Tp { case ast.AlterTableAddColumns: From 65eb3fb386e09a95a74fd7a26e4f57d4c918880f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 10 Mar 2021 01:06:33 +0800 Subject: [PATCH 33/61] Resume the process of drop index --- ddl/db_test.go | 415 ++++++++++++++++++++++---------------------- ddl/ddl_api.go | 99 +++++++---- ddl/ddl_worker.go | 2 +- ddl/delete_range.go | 87 ++++++---- ddl/index.go | 130 +++++++++++++- ddl/rollingback.go | 34 +++- 6 files changed, 490 insertions(+), 277 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 3f05e51e4c34e..804134ea59eb6 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -618,110 +618,109 @@ func (s *testDBSuite4) TestCancelDropPrimaryKey(c *C) { testCancelDropIndex(c, s.store, s.dom.DDL(), idxName, addIdxSQL, dropIdxSQL, s.dom) } -func (s *testDBSuite5) TestCancelDropIndexes(c *C) { - indexesName := []string{"idx_c1", "idx_c2"} - addIdxesSQL := "alter table t add index idx_c1 (c1);alter table t add index idx_c2 (c2);" - dropIdxesSQL := "alter table t drop index idx_c1;alter table t drop index idx_c2;" - - store := s.store - d := s.dom.DDL() - - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists t") - tk.MustExec("create table t(c1 int, c2 int)") - defer tk.MustExec("drop table t;") - for i := 0; i < 5; i++ { - tk.MustExec("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. - // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. - {true, model.JobStateNone, model.StateNone, true}, - {false, model.JobStateRunning, model.StateWriteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteOnly, false}, - {true, model.JobStateRunning, model.StateDeleteReorganization, false}, - } - var checkErr error - hook := &ddl.TestDDLCallback{} - var jobID int64 - testCase := &testCases[0] - hook.OnJobRunBeforeExported = func(job *model.Job) { - if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && - job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { - jobID = job.ID - jobIDs := []int64{job.ID} - hookCtx := mock.NewContext() - hookCtx.Store = store - err := hookCtx.NewTxn(context.TODO()) - if err != nil { - checkErr = errors.Trace(err) - return - } - txn, err := hookCtx.Txn(true) - if err != nil { - checkErr = errors.Trace(err) - return - } - - errs, err := admin.CancelJobs(txn, jobIDs) - if err != nil { - checkErr = errors.Trace(err) - return - } - if errs[0] != nil { - checkErr = errors.Trace(errs[0]) - return - } - checkErr = txn.Commit(context.Background()) - } - } - originalHook := d.GetHook() - d.(ddl.DDLForTest).SetHook(hook) - ctx := tk.Se.(sessionctx.Context) - for i := range testCases { - testCase = &testCases[i] - if testCase.needAddIndex { - tk.MustExec(addIdxesSQL) - } - rs, err := tk.Exec(dropIdxesSQL) - if rs != nil { - rs.Close() - } - t := testGetTableByName(c, ctx, "test_db", "t") - - var indexesInfo []*model.IndexInfo - for _, idxName := range indexesName { - indexInfo := t.Meta().FindIndexByName(idxName) - if indexInfo != nil { - indexesInfo = append(indexesInfo, indexInfo) - } - } - - if testCase.cancelSucc { - c.Assert(checkErr, IsNil) - c.Assert(err, NotNil) - c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") - c.Assert(indexesInfo, NotNil) - c.Assert(indexesInfo[0].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(indexesInfo, IsNil) - } - } - d.(ddl.DDLForTest).SetHook(originalHook) - tk.MustExec(addIdxesSQL) - tk.MustExec(dropIdxesSQL) - -} +// func (s *testDBSuite5) TestCancelDropIndexes(c *C) { +// indexesName := []string{"idx_c1", "idx_c2"} +// addIdxesSQL := "alter table t add index idx_c1 (c1);alter table t add index idx_c2 (c2);" +// dropIdxesSQL := "alter table t drop index idx_c1;alter table t drop index idx_c2;" + +// store := s.store +// d := s.dom.DDL() + +// tk := testkit.NewTestKit(c, store) +// tk.MustExec("use test_db") +// tk.MustExec("drop table if exists t") +// tk.MustExec("create table t(c1 int, c2 int)") +// defer tk.MustExec("drop table t;") +// for i := 0; i < 5; i++ { +// tk.MustExec("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. +// // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. +// {true, model.JobStateNone, model.StateNone, true}, +// {false, model.JobStateRunning, model.StateWriteOnly, false}, +// {true, model.JobStateRunning, model.StateDeleteOnly, false}, +// {true, model.JobStateRunning, model.StateDeleteReorganization, false}, +// } +// var checkErr error +// hook := &ddl.TestDDLCallback{} +// var jobID int64 +// testCase := &testCases[0] +// hook.OnJobRunBeforeExported = func(job *model.Job) { +// if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && +// job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { +// jobID = job.ID +// jobIDs := []int64{job.ID} +// hookCtx := mock.NewContext() +// hookCtx.Store = store +// err := hookCtx.NewTxn(context.TODO()) +// if err != nil { +// checkErr = errors.Trace(err) +// return +// } +// txn, err := hookCtx.Txn(true) +// if err != nil { +// checkErr = errors.Trace(err) +// return +// } + +// errs, err := admin.CancelJobs(txn, jobIDs) +// if err != nil { +// checkErr = errors.Trace(err) +// return +// } +// if errs[0] != nil { +// checkErr = errors.Trace(errs[0]) +// return +// } +// checkErr = txn.Commit(context.Background()) +// } +// } +// originalHook := d.GetHook() +// d.(ddl.DDLForTest).SetHook(hook) +// ctx := tk.Se.(sessionctx.Context) +// for i := range testCases { +// testCase = &testCases[i] +// if testCase.needAddIndex { +// tk.MustExec(addIdxesSQL) +// } +// rs, err := tk.Exec(dropIdxesSQL) +// if rs != nil { +// rs.Close() +// } +// t := testGetTableByName(c, ctx, "test_db", "t") + +// var indexesInfo []*model.IndexInfo +// for _, idxName := range indexesName { +// indexInfo := t.Meta().FindIndexByName(idxName) +// if indexInfo != nil { +// indexesInfo = append(indexesInfo, indexInfo) +// } +// } + +// if testCase.cancelSucc { +// c.Assert(checkErr, IsNil) +// c.Assert(err, NotNil) +// c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") +// c.Assert(indexesInfo, NotNil) +// c.Assert(indexesInfo[0].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(indexesInfo, IsNil) +// } +// } +// d.(ddl.DDLForTest).SetHook(originalHook) +// tk.MustExec(addIdxesSQL) +// tk.MustExec(dropIdxesSQL) +// } // TestCancelDropIndex tests cancel ddl job which type is drop index. func (s *testDBSuite5) TestCancelDropIndex(c *C) { @@ -1596,108 +1595,108 @@ LOOP: tk.MustExec("drop table test_drop_index") } -func (s *testDBSuite2) TestDropDuplicateIndexes(c *C) { - tk := testkit.NewTestKitWithInit(c, s.store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists test_drop_duplicate_indexes;") - tk.MustExec("create table test_drop_duplicate_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") - - if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index i1, drop index i1;"); true { - c.Assert(err.Error(), Equals, "[ddl:1091]index i1 doesn't exist") - } - - tk.MustExec("alter table test_drop_duplicate_indexes drop index i1, drop index if exists i1;") - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index i1 doesn't exist")) - - if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index if exists i2, drop index i2;"); true { - c.Assert(err.Error(), Equals, "[ddl:1091]index i2 doesn't exist") - } - - tk.MustExec("drop table if exists test_drop_duplicate_indexes") -} - -func (s *testDBSuite2) TestDropIndexes(c *C) { - // drop multiple indexes - createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2))" - dropIdxSQL := "alter table test_drop_indexes drop index i1, drop index i2;" - idxNames := []string{"i1", "i2"} - testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) - // test drop primary key and index - dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" - idxNames = []string{"primary", "i1"} - testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) -} - -func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dropIdxSQL string, idxNames []string) { - tk := testkit.NewTestKit(c, store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists test_drop_indexes") - tk.MustExec(createSQL) - done := make(chan error, 1) - tk.MustExec("delete from test_drop_indexes") - - num := 100 - // add some rows - for i := 0; i < num; i++ { - tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) - } - ctx := tk.Se.(sessionctx.Context) - t := testGetTableByName(c, ctx, "test_db", "test_drop_indexes") - var idxs []table.Index - for _, tidx := range t.Indices() { - for _, idxName := range idxNames { - if tidx.Meta().Name.L == idxName { - idxs = append(idxs, tidx) - break - } - } - } - c.Assert(idxs, NotNil) - - testddlutil.SessionExecInGoroutine(c, store, dropIdxSQL, done) - - ticker := time.NewTicker(lease / 2) - defer ticker.Stop() -LOOP: - for { - select { - case err := <-done: - if err == nil { - break LOOP - } - c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) - case <-ticker.C: - step := 5 - // delete some rows, and add some data - for i := num; i < num+step; i++ { - n := rand.Intn(num) - tk.MustExec("update test_drop_indexes set c2 = 1 where c1 = ?", n) - tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) - } - num += step - } - } - - // Check in index, it must be no index in KV. - // Make sure there is no index with name c2_index、c3_index. - t = testGetTableByName(c, ctx, "test_db", "test_drop_indexes") - var nidxs []table.Index - for _, tidx := range t.Indices() { - for _, ids := range idxs { - if tidx.Meta().Name.L == ids.Meta().Name.L { - nidxs = append(nidxs, tidx) - } - } - } - c.Assert(nidxs, IsNil) - - for _, idx := range idxs { - idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) - checkDelRangeDone(c, ctx, idx) - } - - tk.MustExec("drop table test_drop_indexes") -} +// func (s *testDBSuite2) TestDropDuplicateIndexes(c *C) { +// tk := testkit.NewTestKitWithInit(c, s.store) +// tk.MustExec("use test_db") +// tk.MustExec("drop table if exists test_drop_duplicate_indexes;") +// tk.MustExec("create table test_drop_duplicate_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") + +// if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index i1, drop index i1;"); true { +// c.Assert(err.Error(), Equals, "[ddl:1091]index i1 doesn't exist") +// } + +// tk.MustExec("alter table test_drop_duplicate_indexes drop index i1, drop index if exists i1;") +// tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index i1 doesn't exist")) + +// if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index if exists i2, drop index i2;"); true { +// c.Assert(err.Error(), Equals, "[ddl:1091]index i2 doesn't exist") +// } + +// tk.MustExec("drop table if exists test_drop_duplicate_indexes") +// } + +// func (s *testDBSuite2) TestDropIndexes(c *C) { +// // drop multiple indexes +// createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2))" +// dropIdxSQL := "alter table test_drop_indexes drop index i1, drop index i2;" +// idxNames := []string{"i1", "i2"} +// testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) +// // test drop primary key and index +// dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" +// idxNames = []string{"primary", "i1"} +// testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) +// } + +// func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dropIdxSQL string, idxNames []string) { +// tk := testkit.NewTestKit(c, store) +// tk.MustExec("use test_db") +// tk.MustExec("drop table if exists test_drop_indexes") +// tk.MustExec(createSQL) +// done := make(chan error, 1) +// tk.MustExec("delete from test_drop_indexes") + +// num := 100 +// // add some rows +// for i := 0; i < num; i++ { +// tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) +// } +// ctx := tk.Se.(sessionctx.Context) +// t := testGetTableByName(c, ctx, "test_db", "test_drop_indexes") +// var idxs []table.Index +// for _, tidx := range t.Indices() { +// for _, idxName := range idxNames { +// if tidx.Meta().Name.L == idxName { +// idxs = append(idxs, tidx) +// break +// } +// } +// } +// c.Assert(idxs, NotNil) + +// testddlutil.SessionExecInGoroutine(c, store, dropIdxSQL, done) + +// ticker := time.NewTicker(lease / 2) +// defer ticker.Stop() +// LOOP: +// for { +// select { +// case err := <-done: +// if err == nil { +// break LOOP +// } +// c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) +// case <-ticker.C: +// step := 5 +// // delete some rows, and add some data +// for i := num; i < num+step; i++ { +// n := rand.Intn(num) +// tk.MustExec("update test_drop_indexes set c2 = 1 where c1 = ?", n) +// tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) +// } +// num += step +// } +// } + +// // Check in index, it must be no index in KV. +// // Make sure there is no index with name c2_index、c3_index. +// t = testGetTableByName(c, ctx, "test_db", "test_drop_indexes") +// var nidxs []table.Index +// for _, tidx := range t.Indices() { +// for _, ids := range idxs { +// if tidx.Meta().Name.L == ids.Meta().Name.L { +// nidxs = append(nidxs, tidx) +// } +// } +// } +// c.Assert(nidxs, IsNil) + +// for _, idx := range idxs { +// idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) +// checkDelRangeDone(c, ctx, idx) +// } + +// tk.MustExec("drop table test_drop_indexes") +// } // TestCancelDropColumn tests cancel ddl job which type is drop column. func (s *testDBSuite3) TestCancelDropColumn(c *C) { @@ -4701,8 +4700,8 @@ func (s *testDBSuite4) TestIfExists(c *C) { tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index idx_c doesn't exist")) // DROP INDEXES - s.mustExec(tk, c, "alter table t1 drop index if exists idxes_a,drop index if exists idxes_b") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) + // s.mustExec(tk, c, "alter table t1 drop index if exists idxes_a,drop index if exists idxes_b") + // c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) // DROP PARTITION s.mustExec(tk, c, "drop table if exists t2") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index efa2f6cb1446e..b57c09cee34cc 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5166,22 +5166,19 @@ func (d *ddl) DropForeignKey(ctx sessionctx.Context, ti ast.Ident, fkName model. return errors.Trace(err) } -func (d *ddl) getSchemaAndTable(ti ast.Ident) (*model.DBInfo, table.Table, error) { +func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, ifExists bool) error { is := d.infoHandle.Get() schema, ok := is.SchemaByName(ti.Schema) if !ok { - return nil, nil, errors.Trace(infoschema.ErrDatabaseNotExists) + return errors.Trace(infoschema.ErrDatabaseNotExists) } t, err := is.TableByName(ti.Schema, ti.Name) if err != nil { - return nil, nil, errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) } - return schema, t, nil -} -func checkIndexInfo(ctx sessionctx.Context, t table.Table, indexName model.CIStr, ifExists bool) (isPK bool, shouldIgnore bool, _ error) { indexInfo := t.Meta().FindIndexByName(indexName.L) - + var isPK bool if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && // Before we fixed #14243, there might be a general index named `primary` but not a primary key. (indexInfo == nil || indexInfo.Primary) { @@ -5189,48 +5186,33 @@ func checkIndexInfo(ctx sessionctx.Context, t table.Table, indexName model.CIStr } if isPK { if !config.GetGlobalConfig().AlterPrimaryKey { - return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") + } // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. if indexInfo == nil && !t.Meta().PKIsHandle { - return isPK, shouldIgnore, ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") + return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") } if t.Meta().PKIsHandle { - return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") } if t.Meta().IsCommonHandle { - return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") } } - if indexInfo == nil { - err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + err = ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) if ifExists { ctx.GetSessionVars().StmtCtx.AppendNote(err) - return isPK, true, nil + return nil } - return isPK, shouldIgnore, err - } - - // Check for drop index on auto_increment column. - if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { - return isPK, shouldIgnore, errors.Trace(err) - } - return isPK, shouldIgnore, nil -} - -func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CIStr, ifExists bool) error { - schema, t, err := d.getSchemaAndTable(ti) - if err != nil { return err } - isPK, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, ifExists) + // Check for drop index on auto_increment column. + err = checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo) if err != nil { - return err - } - if shouldIgnore { - return nil + return errors.Trace(err) } jobTp := model.ActionDropIndex @@ -5257,6 +5239,59 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI return errors.Trace(err) } +func (d *ddl) getSchemaAndTable(ti ast.Ident) (*model.DBInfo, table.Table, error) { + is := d.infoHandle.Get() + schema, ok := is.SchemaByName(ti.Schema) + if !ok { + return nil, nil, errors.Trace(infoschema.ErrDatabaseNotExists) + } + t, err := is.TableByName(ti.Schema, ti.Name) + if err != nil { + return nil, nil, errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) + } + return schema, t, nil +} + +func checkIndexInfo(ctx sessionctx.Context, t table.Table, indexName model.CIStr, ifExists bool) (isPK bool, shouldIgnore bool, _ error) { + indexInfo := t.Meta().FindIndexByName(indexName.L) + + if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && + // Before we fixed #14243, there might be a general index named `primary` but not a primary key. + (indexInfo == nil || indexInfo.Primary) { + isPK = true + } + if isPK { + if !config.GetGlobalConfig().AlterPrimaryKey { + return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") + } + // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. + if indexInfo == nil && !t.Meta().PKIsHandle { + return isPK, shouldIgnore, ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") + } + if t.Meta().PKIsHandle { + return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") + } + if t.Meta().IsCommonHandle { + return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + } + } + + if indexInfo == nil { + err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + if ifExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + return isPK, true, nil + } + return isPK, shouldIgnore, err + } + + // Check for drop index on auto_increment column. + if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { + return isPK, shouldIgnore, errors.Trace(err) + } + return isPK, shouldIgnore, nil +} + func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.AlterTableSpec) error { schema, t, err := d.getSchemaAndTable(ti) if err != nil { diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index f112ce8c1ce5c..c1fd476ba6801 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -692,7 +692,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionAddPrimaryKey: ver, err = w.onCreateIndex(d, t, job, true) case model.ActionDropIndex, model.ActionDropPrimaryKey: - ver, err = onDropIndexes(t, job) + ver, err = onDropIndex(t, job) case model.ActionRenameIndex: ver, err = onRenameIndex(t, job) case model.ActionAddForeignKey: diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 72a95d75efc02..c0f1655dda298 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -304,51 +304,70 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error // ActionAddIndex, ActionAddPrimaryKey needs do it, because it needs to be rolled back when it's canceled. case model.ActionAddIndex, model.ActionAddPrimaryKey: tableID := job.TableID - var indexIDs []int64 + var indexID int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexIDs, &partitionIDs); err != nil { + if err := job.DecodeArgs(&indexID, &partitionIDs); err != nil { return errors.Trace(err) } - for _, indexID := range indexIDs { - if len(partitionIDs) > 0 { - for _, pid := range partitionIDs { - startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) - if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { - return errors.Trace(err) - } + if len(partitionIDs) > 0 { + for _, pid := range partitionIDs { + startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) + if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + return errors.Trace(err) } - } else { - startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) - endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) - return doInsert(s, job.ID, indexID, startKey, endKey, now) } + } else { + startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + return doInsert(s, job.ID, indexID, startKey, endKey, now) } case model.ActionDropIndex, model.ActionDropPrimaryKey: - var indexNames []model.CIStr - var ifExists []bool - var indexIDs []int64 + tableID := job.TableID + var indexName interface{} + var indexID int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexNames, &ifExists, &indexIDs, &partitionIDs); err != nil { - // Compatible with the previous function: DropIndex - var indexName model.CIStr - if err := job.DecodeArgs(&indexName, &indexIDs, &partitionIDs); err != nil { - return errors.Trace(err) - } - indexNames = append(indexNames, indexName) - } - // Remove data in TiKV. - if len(indexIDs) == 0 { - return nil - } - if len(partitionIDs) == 0 { - return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) + if err := job.DecodeArgs(&indexName, &indexID, &partitionIDs); err != nil { + return errors.Trace(err) } - for _, pid := range partitionIDs { - if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { - return errors.Trace(err) + if len(partitionIDs) > 0 { + for _, pid := range partitionIDs { + startKey := tablecodec.EncodeTableIndexPrefix(pid, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(pid, indexID+1) + if err := doInsert(s, job.ID, indexID, startKey, endKey, now); err != nil { + return errors.Trace(err) + } } + } else { + startKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID) + endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) + return doInsert(s, job.ID, indexID, startKey, endKey, now) } + // case model.ActionDropIndexes: + // var indexNames []model.CIStr + // var ifExists []bool + // var indexIDs []int64 + // var partitionIDs []int64 + // if err := job.DecodeArgs(&indexNames, &ifExists, &indexIDs, &partitionIDs); err != nil { + // // Compatible with the previous function: DropIndex + // var indexName model.CIStr + // if err := job.DecodeArgs(&indexName, &indexIDs, &partitionIDs); err != nil { + // return errors.Trace(err) + // } + // indexNames = append(indexNames, indexName) + // } + // // Remove data in TiKV. + // if len(indexIDs) == 0 { + // return nil + // } + // if len(partitionIDs) == 0 { + // return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) + // } + // for _, pid := range partitionIDs { + // if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { + // return errors.Trace(err) + // } + // } case model.ActionDropColumn: var colName model.CIStr var indexIDs []int64 diff --git a/ddl/index.go b/ddl/index.go index f1f144cada349..40e6b27a9dc04 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -390,7 +390,7 @@ func updateHiddenColumns(tblInfo *model.TableInfo, idxInfo *model.IndexInfo, sta func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK bool) (ver int64, err error) { // Handle the rolling back job. if job.IsRollingback() { - ver, err = onDropIndexes(t, job) + ver, err = onDropIndex(t, job) if err != nil { return ver, errors.Trace(err) } @@ -608,6 +608,134 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo return ver, errors.Trace(err) } +func onDropIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) { + tblInfo, indexInfo, err := checkDropIndex(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + dependentHiddenCols := make([]*model.ColumnInfo, 0) + for _, indexColumn := range indexInfo.Columns { + if tblInfo.Columns[indexColumn.Offset].Hidden { + dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) + } + } + + originalState := indexInfo.State + switch indexInfo.State { + case model.StatePublic: + // public -> write only + indexInfo.State = model.StateWriteOnly + if len(dependentHiddenCols) > 0 { + firstHiddenOffset := dependentHiddenCols[0].Offset + for i := 0; i < len(dependentHiddenCols); i++ { + tblInfo.Columns[firstHiddenOffset].State = model.StateWriteOnly + // Set this column's offset to the last and reset all following columns' offsets. + adjustColumnInfoInDropColumn(tblInfo, firstHiddenOffset) + } + } + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateWriteOnly + case model.StateWriteOnly: + // write only -> delete only + indexInfo.State = model.StateDeleteOnly + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteOnly) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateDeleteOnly + case model.StateDeleteOnly: + // delete only -> reorganization + indexInfo.State = model.StateDeleteReorganization + updateHiddenColumns(tblInfo, indexInfo, model.StateDeleteReorganization) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) + if err != nil { + return ver, errors.Trace(err) + } + job.SchemaState = model.StateDeleteReorganization + case model.StateDeleteReorganization: + // reorganization -> absent + 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) + + tblInfo.Columns = tblInfo.Columns[:len(tblInfo.Columns)-len(dependentHiddenCols)] + + ver, err = updateVersionAndTableInfoWithCheck(t, job, tblInfo, originalState != model.StateNone) + if err != nil { + return ver, errors.Trace(err) + } + + // Finish this job. + if job.IsRollingback() { + job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) + job.Args[0] = indexInfo.ID + // the partition ids were append by convertAddIdxJob2RollbackJob, it is weird, but for the compatibility, + // we should keep appending the partitions in the convertAddIdxJob2RollbackJob. + } else { + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + job.Args = append(job.Args, indexInfo.ID, getPartitionIDs(tblInfo)) + } + default: + err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfo.State) + } + return ver, errors.Trace(err) +} + +func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.IndexInfo, error) { + schemaID := job.SchemaID + tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID) + if err != nil { + return nil, nil, errors.Trace(err) + } + + var indexName model.CIStr + if err = job.DecodeArgs(&indexName); err != nil { + job.State = model.JobStateCancelled + return nil, nil, errors.Trace(err) + } + + indexInfo := tblInfo.FindIndexByName(indexName.L) + if indexInfo == nil { + job.State = model.JobStateCancelled + return nil, nil, 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 + } + + // Check that drop primary index will not cause invisible implicit primary index. + newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) + for _, idx := range tblInfo.Indices { + if idx.Name.L != indexInfo.Name.L { + newIndices = append(newIndices, idx) + } + } + newTbl := tblInfo.Clone() + newTbl.Indices = newIndices + err = checkInvisibleIndexOnPK(newTbl) + if err != nil { + job.State = model.JobStateCancelled + return nil, nil, errors.Trace(err) + } + + return tblInfo, indexInfo, nil +} + func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo, idxInfos, err := checkDropIndexes(t, job) if err != nil { diff --git a/ddl/rollingback.go b/ddl/rollingback.go index bfa945079a66c..8b83875487a86 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -255,6 +255,36 @@ func rollingbackDropColumns(t *meta.Meta, job *model.Job) (ver int64, err error) return ver, nil } +func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { + tblInfo, indexInfo, err := checkDropIndex(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + originalState := indexInfo.State + switch indexInfo.State { + case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: + // 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: + job.State = model.JobStateRollbackDone + indexInfo.State = model.StatePublic + default: + return ver, ErrInvalidDDLState.GenWithStackByArgs("index", 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) + } + job.FinishTableJob(model.JobStateRollbackDone, model.StatePublic, ver, tblInfo) + return ver, errCancelledDDLJob +} + func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) { tblInfo, indexesInfo, err := checkDropIndexes(t, job) if err != nil { @@ -425,7 +455,9 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) case model.ActionDropColumns: ver, err = rollingbackDropColumns(t, job) case model.ActionDropIndex, model.ActionDropPrimaryKey: - ver, err = rollingbackDropIndexes(t, job) + ver, err = rollingbackDropIndex(t, job) + // case model.ActionDropIndexes: + // ver, err = rollingbackDropIndexes(t, job) case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: err = rollingbackDropTableOrView(t, job) case model.ActionDropTablePartition: From 9a32310f7bd386ed3af1ff05cd97e56cd89b730c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 10 Mar 2021 10:01:38 +0800 Subject: [PATCH 34/61] Resume the process of drop index --- ddl/ddl_api.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index b57c09cee34cc..cb7d7ad4e1b19 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2352,12 +2352,12 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return errRunMultiSchemaChanges } - if isDropIndexes(validSpecs) { - if err = d.DropIndexes(ctx, ident, validSpecs); err != nil { - return errors.Trace(err) - } - return nil - } + // if isDropIndexes(validSpecs) { + // if err = d.DropIndexes(ctx, ident, validSpecs); err != nil { + // return errors.Trace(err) + // } + // return nil + // } if isSameTypeMultiSpecs(validSpecs) { switch validSpecs[0].Tp { From a9cef9ea49dc5bb553e8fe62a3cf951b51e81b30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 10 Mar 2021 21:00:56 +0800 Subject: [PATCH 35/61] typo --- ddl/db_test.go | 10 +++++----- ddl/index.go | 20 ++++++++++---------- ddl/rollingback.go | 8 ++++---- 3 files changed, 19 insertions(+), 19 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 804134ea59eb6..f81776e77998c 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -695,11 +695,11 @@ func (s *testDBSuite4) TestCancelDropPrimaryKey(c *C) { // } // t := testGetTableByName(c, ctx, "test_db", "t") -// var indexesInfo []*model.IndexInfo +// var indexInfos []*model.IndexInfo // for _, idxName := range indexesName { // indexInfo := t.Meta().FindIndexByName(idxName) // if indexInfo != nil { -// indexesInfo = append(indexesInfo, indexInfo) +// indexInfos = append(indexInfos, indexInfo) // } // } @@ -707,14 +707,14 @@ func (s *testDBSuite4) TestCancelDropPrimaryKey(c *C) { // c.Assert(checkErr, IsNil) // c.Assert(err, NotNil) // c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") -// c.Assert(indexesInfo, NotNil) -// c.Assert(indexesInfo[0].State, Equals, model.StatePublic) +// c.Assert(indexInfos, NotNil) +// c.Assert(indexInfos[0].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(indexesInfo, IsNil) +// c.Assert(indexInfos, IsNil) // } // } // d.(ddl.DDLForTest).SetHook(originalHook) diff --git a/ddl/index.go b/ddl/index.go index 40e6b27a9dc04..56162ddb953f7 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -789,16 +789,16 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.SchemaState = model.StateDeleteReorganization case model.StateDeleteReorganization: // reorganization -> absent - indexesIDs := make([]int64, 0, len(idxInfos)) - indexesLName := make(map[string]bool, len(idxInfos)) + indexIDs := make([]int64, 0, len(idxInfos)) + indexNames := make(map[string]bool, len(idxInfos)) for _, indexInfo := range idxInfos { - indexesLName[indexInfo.Name.L] = true - indexesIDs = append(indexesIDs, indexInfo.ID) + indexNames[indexInfo.Name.L] = true + indexIDs = append(indexIDs, indexInfo.ID) } newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) for _, idx := range tblInfo.Indices { - if _, ok := indexesLName[idx.Name.L]; !ok { + if _, ok := indexNames[idx.Name.L]; !ok { newIndices = append(newIndices, idx) } } @@ -819,12 +819,12 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { // Finish this job. if job.IsRollingback() { job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) - job.Args[0] = indexesIDs + job.Args[0] = indexIDs // the partition ids were append by convertAddIdxJob2RollbackJob, it is weird, but for the compatibility, // we should keep appending the partitions in the convertAddIdxJob2RollbackJob. } else { job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - job.Args = append(job.Args, indexesIDs, getPartitionIDs(tblInfo)) + job.Args = append(job.Args, indexIDs, getPartitionIDs(tblInfo)) } default: err = ErrInvalidDDLState.GenWithStackByArgs("index", idxInfos[0].State) @@ -852,7 +852,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. ifExists = append(ifExists, false) } - indexesInfo := make([]*model.IndexInfo, 0, len(indexNames)) + indexInfos := make([]*model.IndexInfo, 0, len(indexNames)) droppedIndexes := make(map[string]bool, len(indexNames)) for i, indexName := range indexNames { // Double check the index is exists @@ -870,7 +870,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. job.State = model.JobStateCancelled return nil, nil, autoid.ErrWrongAutoKey } - indexesInfo = append(indexesInfo, indexInfo) + indexInfos = append(indexInfos, indexInfo) droppedIndexes[indexName.L] = true } // Check that drop primary index will not cause invisible implicit primary index. @@ -887,7 +887,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. job.State = model.JobStateCancelled return nil, nil, errors.Trace(err) } - return tblInfo, indexesInfo, nil + return tblInfo, indexInfos, nil } func checkDropIndexOnAutoIncrementColumn(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 8b83875487a86..33caf6637d5b8 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -286,12 +286,12 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { } func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) { - tblInfo, indexesInfo, err := checkDropIndexes(t, job) + tblInfo, indexInfos, err := checkDropIndexes(t, job) if err != nil { return ver, errors.Trace(err) } - indexInfo := indexesInfo[0] + indexInfo := indexInfos[0] originalState := indexInfo.State switch indexInfo.State { case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: @@ -301,7 +301,7 @@ func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) return ver, nil case model.StatePublic: job.State = model.JobStateRollbackDone - for _, indexInfo := range indexesInfo { + for _, indexInfo := range indexInfos { indexInfo.State = model.StatePublic } default: @@ -310,7 +310,7 @@ func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) job.SchemaState = indexInfo.State var indexNames []model.CIStr - for _, indexInfo := range indexesInfo { + for _, indexInfo := range indexInfos { indexNames = append(indexNames, indexInfo.Name) } job.Args = []interface{}{indexNames} From 75fa9a0cdf83b5d411694403dedc2b53a0693bf4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Fri, 19 Mar 2021 23:58:40 +0800 Subject: [PATCH 36/61] The process of modifying drop --- ddl/db_test.go | 424 ++++++++++++++++++++++---------------------- ddl/ddl_api.go | 114 ++++++------ ddl/ddl_worker.go | 4 +- ddl/delete_range.go | 45 +++-- ddl/index.go | 52 +++--- ddl/rollingback.go | 4 +- 6 files changed, 314 insertions(+), 329 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 35269d94096db..c9c3ff125f279 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -617,110 +617,6 @@ func (s *testDBSuite4) TestCancelDropPrimaryKey(c *C) { testCancelDropIndex(c, s.store, s.dom.DDL(), idxName, addIdxSQL, dropIdxSQL, s.dom) } -// func (s *testDBSuite5) TestCancelDropIndexes(c *C) { -// indexesName := []string{"idx_c1", "idx_c2"} -// addIdxesSQL := "alter table t add index idx_c1 (c1);alter table t add index idx_c2 (c2);" -// dropIdxesSQL := "alter table t drop index idx_c1;alter table t drop index idx_c2;" - -// store := s.store -// d := s.dom.DDL() - -// tk := testkit.NewTestKit(c, store) -// tk.MustExec("use test_db") -// tk.MustExec("drop table if exists t") -// tk.MustExec("create table t(c1 int, c2 int)") -// defer tk.MustExec("drop table t;") -// for i := 0; i < 5; i++ { -// tk.MustExec("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. -// // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. -// {true, model.JobStateNone, model.StateNone, true}, -// {false, model.JobStateRunning, model.StateWriteOnly, false}, -// {true, model.JobStateRunning, model.StateDeleteOnly, false}, -// {true, model.JobStateRunning, model.StateDeleteReorganization, false}, -// } -// var checkErr error -// hook := &ddl.TestDDLCallback{} -// var jobID int64 -// testCase := &testCases[0] -// hook.OnJobRunBeforeExported = func(job *model.Job) { -// if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && -// job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { -// jobID = job.ID -// jobIDs := []int64{job.ID} -// hookCtx := mock.NewContext() -// hookCtx.Store = store -// err := hookCtx.NewTxn(context.TODO()) -// if err != nil { -// checkErr = errors.Trace(err) -// return -// } -// txn, err := hookCtx.Txn(true) -// if err != nil { -// checkErr = errors.Trace(err) -// return -// } - -// errs, err := admin.CancelJobs(txn, jobIDs) -// if err != nil { -// checkErr = errors.Trace(err) -// return -// } -// if errs[0] != nil { -// checkErr = errors.Trace(errs[0]) -// return -// } -// checkErr = txn.Commit(context.Background()) -// } -// } -// originalHook := d.GetHook() -// d.(ddl.DDLForTest).SetHook(hook) -// ctx := tk.Se.(sessionctx.Context) -// for i := range testCases { -// testCase = &testCases[i] -// if testCase.needAddIndex { -// tk.MustExec(addIdxesSQL) -// } -// rs, err := tk.Exec(dropIdxesSQL) -// if rs != nil { -// rs.Close() -// } -// t := testGetTableByName(c, ctx, "test_db", "t") - -// var indexInfos []*model.IndexInfo -// for _, idxName := range indexesName { -// indexInfo := t.Meta().FindIndexByName(idxName) -// if indexInfo != nil { -// indexInfos = append(indexInfos, indexInfo) -// } -// } - -// if testCase.cancelSucc { -// c.Assert(checkErr, IsNil) -// c.Assert(err, NotNil) -// c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") -// c.Assert(indexInfos, NotNil) -// c.Assert(indexInfos[0].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(indexInfos, IsNil) -// } -// } -// d.(ddl.DDLForTest).SetHook(originalHook) -// tk.MustExec(addIdxesSQL) -// tk.MustExec(dropIdxesSQL) -// } - // TestCancelDropIndex tests cancel ddl job which type is drop index. func (s *testDBSuite5) TestCancelDropIndex(c *C) { idxName := "idx_c2" @@ -1598,109 +1494,6 @@ LOOP: tk.MustExec("drop table test_drop_index") } -// func (s *testDBSuite2) TestDropDuplicateIndexes(c *C) { -// tk := testkit.NewTestKitWithInit(c, s.store) -// tk.MustExec("use test_db") -// tk.MustExec("drop table if exists test_drop_duplicate_indexes;") -// tk.MustExec("create table test_drop_duplicate_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") - -// if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index i1, drop index i1;"); true { -// c.Assert(err.Error(), Equals, "[ddl:1091]index i1 doesn't exist") -// } - -// tk.MustExec("alter table test_drop_duplicate_indexes drop index i1, drop index if exists i1;") -// tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index i1 doesn't exist")) - -// if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index if exists i2, drop index i2;"); true { -// c.Assert(err.Error(), Equals, "[ddl:1091]index i2 doesn't exist") -// } - -// tk.MustExec("drop table if exists test_drop_duplicate_indexes") -// } - -// func (s *testDBSuite2) TestDropIndexes(c *C) { -// // drop multiple indexes -// createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2))" -// dropIdxSQL := "alter table test_drop_indexes drop index i1, drop index i2;" -// idxNames := []string{"i1", "i2"} -// testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) -// // test drop primary key and index -// dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" -// idxNames = []string{"primary", "i1"} -// testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) -// } - -// func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dropIdxSQL string, idxNames []string) { -// tk := testkit.NewTestKit(c, store) -// tk.MustExec("use test_db") -// tk.MustExec("drop table if exists test_drop_indexes") -// tk.MustExec(createSQL) -// done := make(chan error, 1) -// tk.MustExec("delete from test_drop_indexes") - -// num := 100 -// // add some rows -// for i := 0; i < num; i++ { -// tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) -// } -// ctx := tk.Se.(sessionctx.Context) -// t := testGetTableByName(c, ctx, "test_db", "test_drop_indexes") -// var idxs []table.Index -// for _, tidx := range t.Indices() { -// for _, idxName := range idxNames { -// if tidx.Meta().Name.L == idxName { -// idxs = append(idxs, tidx) -// break -// } -// } -// } -// c.Assert(idxs, NotNil) - -// testddlutil.SessionExecInGoroutine(c, store, dropIdxSQL, done) - -// ticker := time.NewTicker(lease / 2) -// defer ticker.Stop() -// LOOP: -// for { -// select { -// case err := <-done: -// if err == nil { -// break LOOP -// } -// c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) -// case <-ticker.C: -// step := 5 -// // delete some rows, and add some data -// for i := num; i < num+step; i++ { -// n := rand.Intn(num) -// tk.MustExec("update test_drop_indexes set c2 = 1 where c1 = ?", n) -// tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) -// } -// num += step -// } -// } - -// // Check in index, it must be no index in KV. -// // Make sure there is no index with name c2_index、c3_index. -// t = testGetTableByName(c, ctx, "test_db", "test_drop_indexes") -// var nidxs []table.Index -// for _, tidx := range t.Indices() { -// for _, ids := range idxs { -// if tidx.Meta().Name.L == ids.Meta().Name.L { -// nidxs = append(nidxs, tidx) -// } -// } -// } -// c.Assert(nidxs, IsNil) - -// for _, idx := range idxs { -// idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) -// checkDelRangeDone(c, ctx, idx) -// } - -// tk.MustExec("drop table test_drop_indexes") -// } - // TestCancelDropColumn tests cancel ddl job which type is drop column. func (s *testDBSuite3) TestCancelDropColumn(c *C) { tk := testkit.NewTestKit(c, s.store) @@ -4697,10 +4490,6 @@ func (s *testDBSuite4) TestIfExists(c *C) { c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index idx_c doesn't exist")) - // DROP INDEXES - // s.mustExec(tk, c, "alter table t1 drop index if exists idxes_a,drop index if exists idxes_b") - // c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) - // DROP PARTITION s.mustExec(tk, c, "drop table if exists t2") s.mustExec(tk, c, "create table t2 (a int key) partition by range(a) (partition p0 values less than (10), partition p1 values less than (20))") @@ -4710,6 +4499,10 @@ func (s *testDBSuite4) TestIfExists(c *C) { s.mustExec(tk, c, "alter table t2 drop partition if exists p1") c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1507|Error in list of partitions to p1")) + + // DROP INDEXES + s.mustExec(tk, c, "alter table t1 drop index if exists idxes_a,drop index if exists idxes_b") + c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) } func testAddIndexForGeneratedColumn(tk *testkit.TestKit, s *testSerialDBSuite, c *C) { @@ -6821,3 +6614,212 @@ func (s *testSerialDBSuite) TestIssue22819(c *C) { _, err := tk1.Exec("commit") c.Assert(err, ErrorMatches, ".*8028.*Information schema is changed during the execution of the statement.*") } + +func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dropIdxSQL string, idxNames []string) { + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists test_drop_indexes") + tk.MustExec(createSQL) + done := make(chan error, 1) + tk.MustExec("delete from test_drop_indexes") + + num := 100 + // add some rows + for i := 0; i < num; i++ { + tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) + } + ctx := tk.Se.(sessionctx.Context) + t := testGetTableByName(c, ctx, "test_db", "test_drop_indexes") + var idxs []table.Index + for _, tidx := range t.Indices() { + for _, idxName := range idxNames { + if tidx.Meta().Name.L == idxName { + idxs = append(idxs, tidx) + break + } + } + } + c.Assert(idxs, NotNil) + + testddlutil.SessionExecInGoroutine(c, store, dropIdxSQL, done) + + ticker := time.NewTicker(lease / 2) + defer ticker.Stop() +LOOP: + for { + select { + case err := <-done: + if err == nil { + break LOOP + } + c.Assert(err, IsNil, Commentf("err:%v", errors.ErrorStack(err))) + case <-ticker.C: + step := 5 + // delete some rows, and add some data + for i := num; i < num+step; i++ { + n := rand.Intn(num) + tk.MustExec("update test_drop_indexes set c2 = 1 where c1 = ?", n) + tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) + } + num += step + } + } + + // Check in index, it must be no index in KV. + // Make sure there is no index with name c2_index、c3_index. + t = testGetTableByName(c, ctx, "test_db", "test_drop_indexes") + var nidxs []table.Index + for _, tidx := range t.Indices() { + for _, ids := range idxs { + if tidx.Meta().Name.L == ids.Meta().Name.L { + nidxs = append(nidxs, tidx) + } + } + } + c.Assert(nidxs, IsNil) + + for _, idx := range idxs { + idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) + checkDelRangeDone(c, ctx, idx) + } + + tk.MustExec("drop table test_drop_indexes") +} + +func testDropDuplicateIndexes(c *C, store kv.Storage) { + tk := testkit.NewTestKitWithInit(c, store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists test_drop_duplicate_indexes;") + tk.MustExec("create table test_drop_duplicate_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") + + if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index i1, drop index i1;"); true { + c.Assert(err.Error(), Equals, "[ddl:1091]index i1 doesn't exist") + } + + tk.MustExec("alter table test_drop_duplicate_indexes drop index i1, drop index if exists i1;") + tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index i1 doesn't exist")) + + if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index if exists i2, drop index i2;"); true { + c.Assert(err.Error(), Equals, "[ddl:1091]index i2 doesn't exist") + } + + tk.MustExec("drop table if exists test_drop_duplicate_indexes") +} + +func testCancelDropIndexes(c *C, store kv.Storage, d ddl.DDL) { + indexesName := []string{"idx_c1", "idx_c2"} + addIdxesSQL := "alter table t add index idx_c1 (c1);alter table t add index idx_c2 (c2);" + dropIdxesSQL := "alter table t drop index idx_c1;alter table t drop index idx_c2;" + + tk := testkit.NewTestKit(c, store) + tk.MustExec("use test_db") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(c1 int, c2 int)") + defer tk.MustExec("drop table t;") + for i := 0; i < 5; i++ { + tk.MustExec("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. + // if we cancel successfully, we need to set needAddIndex to false in the next test case. Otherwise, set needAddIndex to true. + {true, model.JobStateNone, model.StateNone, true}, + {false, model.JobStateRunning, model.StateWriteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteOnly, false}, + {true, model.JobStateRunning, model.StateDeleteReorganization, false}, + } + var checkErr error + hook := &ddl.TestDDLCallback{} + var jobID int64 + testCase := &testCases[0] + hook.OnJobRunBeforeExported = func(job *model.Job) { + if (job.Type == model.ActionDropIndex || job.Type == model.ActionDropPrimaryKey) && + job.State == testCase.jobState && job.SchemaState == testCase.JobSchemaState { + jobID = job.ID + jobIDs := []int64{job.ID} + hookCtx := mock.NewContext() + hookCtx.Store = store + err := hookCtx.NewTxn(context.TODO()) + if err != nil { + checkErr = errors.Trace(err) + return + } + txn, err := hookCtx.Txn(true) + if err != nil { + checkErr = errors.Trace(err) + return + } + + errs, err := admin.CancelJobs(txn, jobIDs) + if err != nil { + checkErr = errors.Trace(err) + return + } + if errs[0] != nil { + checkErr = errors.Trace(errs[0]) + return + } + checkErr = txn.Commit(context.Background()) + } + } + originalHook := d.GetHook() + d.(ddl.DDLForTest).SetHook(hook) + ctx := tk.Se.(sessionctx.Context) + for i := range testCases { + testCase = &testCases[i] + if testCase.needAddIndex { + tk.MustExec(addIdxesSQL) + } + rs, err := tk.Exec(dropIdxesSQL) + if rs != nil { + rs.Close() + } + t := testGetTableByName(c, ctx, "test_db", "t") + + var indexInfos []*model.IndexInfo + for _, idxName := range indexesName { + indexInfo := t.Meta().FindIndexByName(idxName) + if indexInfo != nil { + indexInfos = append(indexInfos, indexInfo) + } + } + + if testCase.cancelSucc { + c.Assert(checkErr, IsNil) + c.Assert(err, NotNil) + c.Assert(err.Error(), Equals, "[ddl:8214]Cancelled DDL job") + c.Assert(indexInfos, NotNil) + c.Assert(indexInfos[0].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(indexInfos, IsNil) + } + } + d.(ddl.DDLForTest).SetHook(originalHook) + tk.MustExec(addIdxesSQL) + tk.MustExec(dropIdxesSQL) +} + +func (s *testDBSuite5) TestDropIndexes(c *C) { + // drop multiple indexes + createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));" + dropIdxSQL := "alter table test_drop_indexes drop index i1, drop index i2;" + idxNames := []string{"i1", "i2"} + testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) + + // test drop primary key and index + dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" + idxNames = []string{"primary", "i1"} + testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) + + testDropDuplicateIndexes(c, s.store) + + testCancelDropIndexes(c, s.store, s.dom.DDL()) +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 825152c69f623..3a1d0f3fc4345 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2357,12 +2357,12 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return errRunMultiSchemaChanges } - // if isDropIndexes(validSpecs) { - // if err = d.DropIndexes(ctx, ident, validSpecs); err != nil { - // return errors.Trace(err) - // } - // return nil - // } + if isDropIndexes(validSpecs) { + if err = d.DropIndexes(ctx, ident, validSpecs); err != nil { + return errors.Trace(err) + } + return nil + } if isSameTypeMultiSpecs(validSpecs) { switch validSpecs[0].Tp { @@ -5257,55 +5257,16 @@ func (d *ddl) getSchemaAndTable(ti ast.Ident) (*model.DBInfo, table.Table, error return schema, t, nil } -func checkIndexInfo(ctx sessionctx.Context, t table.Table, indexName model.CIStr, ifExists bool) (isPK bool, shouldIgnore bool, _ error) { - indexInfo := t.Meta().FindIndexByName(indexName.L) - - if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && - // Before we fixed #14243, there might be a general index named `primary` but not a primary key. - (indexInfo == nil || indexInfo.Primary) { - isPK = true - } - if isPK { - if !config.GetGlobalConfig().AlterPrimaryKey { - return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") - } - // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. - if indexInfo == nil && !t.Meta().PKIsHandle { - return isPK, shouldIgnore, ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") - } - if t.Meta().PKIsHandle { - return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") - } - if t.Meta().IsCommonHandle { - return isPK, shouldIgnore, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") - } - } - - if indexInfo == nil { - err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) - if ifExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - return isPK, true, nil - } - return isPK, shouldIgnore, err - } - - // Check for drop index on auto_increment column. - if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { - return isPK, shouldIgnore, errors.Trace(err) - } - return isPK, shouldIgnore, nil -} - func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.AlterTableSpec) error { schema, t, err := d.getSchemaAndTable(ti) if err != nil { return err } - // We need to confirm whether we should return an error when dropping duplicate indexes - indexesIfExists := make(map[string]bool, len(specs)) - droppingIndex := make([]model.CIStr, 0, len(specs)) + uniqueIndex := make(map[string]bool, len(specs)) + indexNames := make([]model.CIStr, 0, len(specs)) + ifExists := make([]bool, 0, len(specs)) + for _, spec := range specs { var indexName model.CIStr if spec.Tp == ast.AlterTableDropPrimaryKey { @@ -5313,7 +5274,8 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt } else { indexName = model.NewCIStr(spec.Name) } - _, ok := indexesIfExists[indexName.L] + + _, ok := uniqueIndex[indexName.L] if ok { err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) if spec.IfExists { @@ -5322,24 +5284,50 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt } return err } - indexesIfExists[indexName.L] = spec.IfExists - droppingIndex = append(droppingIndex, indexName) - } + uniqueIndex[indexName.L] = spec.IfExists - // Check the index is droppable. - indexNames := make([]model.CIStr, 0, len(droppingIndex)) - ifExists := make([]bool, 0, len(droppingIndex)) - for _, indexName := range droppingIndex { - _, shouldIgnore, err := checkIndexInfo(ctx, t, indexName, indexesIfExists[indexName.L]) - if err != nil { + indexInfo := t.Meta().FindIndexByName(indexName.L) + var isPK bool + if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && + // Before we fixed #14243, there might be a general index named `primary` but not a primary key. + (indexInfo == nil || indexInfo.Primary) { + isPK = true + } + if isPK { + if !config.GetGlobalConfig().AlterPrimaryKey { + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") + + } + // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. + if indexInfo == nil && !t.Meta().PKIsHandle { + return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") + } + if t.Meta().PKIsHandle { + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") + } + if t.Meta().IsCommonHandle { + return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + } + } + + if indexInfo == nil { + err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + if spec.IfExists { + ctx.GetSessionVars().StmtCtx.AppendNote(err) + continue + } return err } - if shouldIgnore { - continue + + // Check for drop index on auto_increment column. + if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { + return errors.Trace(err) } + indexNames = append(indexNames, indexName) - ifExists = append(ifExists, indexesIfExists[indexName.L]) + ifExists = append(ifExists, spec.IfExists) } + if len(indexNames) == 0 { return nil } @@ -5348,7 +5336,7 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt SchemaID: schema.ID, TableID: t.Meta().ID, SchemaName: schema.Name.L, - Type: model.ActionDropIndex, + Type: model.ActionDropIndexes, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{indexNames, ifExists}, } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 1c263821b932b..492df304b1697 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -381,7 +381,7 @@ func (w *worker) finishDDLJob(t *meta.Meta, job *model.Job) (err error) { // After rolling back an AddIndex operation, we need to use delete-range to delete the half-done index data. err = w.deleteRange(job) - case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex, model.ActionDropPrimaryKey, + case model.ActionDropSchema, model.ActionDropTable, model.ActionTruncateTable, model.ActionDropIndex, model.ActionDropIndexes, model.ActionDropPrimaryKey, model.ActionDropTablePartition, model.ActionTruncateTablePartition, model.ActionDropColumn, model.ActionDropColumns, model.ActionModifyColumn: err = w.deleteRange(job) } @@ -736,6 +736,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = w.onCreateIndex(d, t, job, true) case model.ActionDropIndex, model.ActionDropPrimaryKey: ver, err = onDropIndex(t, job) + case model.ActionDropIndexes: + ver, err = onDropIndexes(t, job) case model.ActionRenameIndex: ver, err = onRenameIndex(t, job) case model.ActionAddForeignKey: diff --git a/ddl/delete_range.go b/ddl/delete_range.go index c0f1655dda298..d14851c00c9f1 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -343,31 +343,26 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error endKey := tablecodec.EncodeTableIndexPrefix(tableID, indexID+1) return doInsert(s, job.ID, indexID, startKey, endKey, now) } - // case model.ActionDropIndexes: - // var indexNames []model.CIStr - // var ifExists []bool - // var indexIDs []int64 - // var partitionIDs []int64 - // if err := job.DecodeArgs(&indexNames, &ifExists, &indexIDs, &partitionIDs); err != nil { - // // Compatible with the previous function: DropIndex - // var indexName model.CIStr - // if err := job.DecodeArgs(&indexName, &indexIDs, &partitionIDs); err != nil { - // return errors.Trace(err) - // } - // indexNames = append(indexNames, indexName) - // } - // // Remove data in TiKV. - // if len(indexIDs) == 0 { - // return nil - // } - // if len(partitionIDs) == 0 { - // return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) - // } - // for _, pid := range partitionIDs { - // if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { - // return errors.Trace(err) - // } - // } + case model.ActionDropIndexes: + var indexNames []model.CIStr + var ifExists []bool + var indexIDs []int64 + var partitionIDs []int64 + if err := job.DecodeArgs(&indexNames, &ifExists, &indexIDs, &partitionIDs); err != nil { + return errors.Trace(err) + } + // Remove data in TiKV. + if len(indexIDs) == 0 { + return nil + } + if len(partitionIDs) == 0 { + return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) + } + for _, pid := range partitionIDs { + if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { + return errors.Trace(err) + } + } case model.ActionDropColumn: var colName model.CIStr var indexIDs []int64 diff --git a/ddl/index.go b/ddl/index.go index 5c31021a294b5..68cfbdfcfdc40 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -740,17 +740,18 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde } func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { - tblInfo, idxInfos, err := checkDropIndexes(t, job) + tblInfo, indexInfos, err := checkDropIndexes(t, job) if err != nil { return ver, errors.Trace(err) } - if len(idxInfos) == 0 { + + if len(indexInfos) == 0 { job.State = model.JobStateCancelled return ver, nil } dependentHiddenCols := make([]*model.ColumnInfo, 0) - for _, indexInfo := range idxInfos { + for _, indexInfo := range indexInfos { for _, indexColumn := range indexInfo.Columns { if tblInfo.Columns[indexColumn.Offset].Hidden { dependentHiddenCols = append(dependentHiddenCols, tblInfo.Columns[indexColumn.Offset]) @@ -758,43 +759,43 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { } } - originalState := idxInfos[0].State - switch idxInfos[0].State { + originalState := indexInfos[0].State + switch indexInfos[0].State { case model.StatePublic: // public -> write only - setIndicesState(idxInfos, model.StateWriteOnly) + setIndicesState(indexInfos, model.StateWriteOnly) setColumnsState(dependentHiddenCols, model.StateWriteOnly) for _, colInfo := range dependentHiddenCols { adjustColumnInfoInDropColumn(tblInfo, colInfo.Offset) } - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfos[0].State) if err != nil { return ver, errors.Trace(err) } job.SchemaState = model.StateWriteOnly case model.StateWriteOnly: // write only -> delete only - setIndicesState(idxInfos, model.StateDeleteOnly) + setIndicesState(indexInfos, model.StateDeleteOnly) setColumnsState(dependentHiddenCols, model.StateDeleteOnly) - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfos[0].State) if err != nil { return ver, errors.Trace(err) } job.SchemaState = model.StateDeleteOnly case model.StateDeleteOnly: // delete only -> reorganization - setIndicesState(idxInfos, model.StateDeleteReorganization) + setIndicesState(indexInfos, model.StateDeleteReorganization) setColumnsState(dependentHiddenCols, model.StateDeleteReorganization) - ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != idxInfos[0].State) + ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfos[0].State) if err != nil { return ver, errors.Trace(err) } job.SchemaState = model.StateDeleteReorganization case model.StateDeleteReorganization: // reorganization -> absent - indexIDs := make([]int64, 0, len(idxInfos)) - indexNames := make(map[string]bool, len(idxInfos)) - for _, indexInfo := range idxInfos { + indexIDs := make([]int64, 0, len(indexInfos)) + indexNames := make(map[string]bool, len(indexInfos)) + for _, indexInfo := range indexInfos { indexNames[indexInfo.Name.L] = true indexIDs = append(indexIDs, indexInfo.ID) } @@ -808,7 +809,7 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo.Indices = newIndices // Set column index flag. - for _, indexInfo := range idxInfos { + for _, indexInfo := range indexInfos { dropIndexColumnFlag(tblInfo, indexInfo) } @@ -830,7 +831,7 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { job.Args = append(job.Args, indexIDs, getPartitionIDs(tblInfo)) } default: - err = ErrInvalidDDLState.GenWithStackByArgs("index", idxInfos[0].State) + err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfos[0].State) } return ver, errors.Trace(err) } @@ -845,18 +846,12 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. var indexNames []model.CIStr var ifExists []bool if err = job.DecodeArgs(&indexNames, &ifExists); err != nil { - // Compatible with the previous function: DropIndex - var indexName model.CIStr - if err = job.DecodeArgs(&indexName); err != nil { - job.State = model.JobStateCancelled - return nil, nil, errors.Trace(err) - } - indexNames = append(indexNames, indexName) - ifExists = append(ifExists, false) + return nil, nil, errors.Trace(err) } indexInfos := make([]*model.IndexInfo, 0, len(indexNames)) - droppedIndexes := make(map[string]bool, len(indexNames)) + uniqueIndex := make(map[string]bool, len(indexNames)) + for i, indexName := range indexNames { // Double check the index is exists indexInfo := tblInfo.FindIndexByName(indexName.L) @@ -867,6 +862,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. } continue } + // Double check for drop index on auto_increment column. err = checkDropIndexOnAutoIncrementColumn(tblInfo, indexInfo) if err != nil { @@ -874,12 +870,13 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return nil, nil, autoid.ErrWrongAutoKey } indexInfos = append(indexInfos, indexInfo) - droppedIndexes[indexName.L] = true + uniqueIndex[indexName.L] = true } + // Check that drop primary index will not cause invisible implicit primary index. newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) for _, idx := range tblInfo.Indices { - if _, ok := droppedIndexes[idx.Name.L]; !ok { + if _, ok := uniqueIndex[idx.Name.L]; !ok { newIndices = append(newIndices, idx) } } @@ -890,6 +887,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. job.State = model.JobStateCancelled return nil, nil, errors.Trace(err) } + return tblInfo, indexInfos, nil } diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 33caf6637d5b8..3dc0980fcaabf 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -456,8 +456,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) ver, err = rollingbackDropColumns(t, job) case model.ActionDropIndex, model.ActionDropPrimaryKey: ver, err = rollingbackDropIndex(t, job) - // case model.ActionDropIndexes: - // ver, err = rollingbackDropIndexes(t, job) + case model.ActionDropIndexes: + ver, err = rollingbackDropIndexes(t, job) case model.ActionDropTable, model.ActionDropView, model.ActionDropSequence: err = rollingbackDropTableOrView(t, job) case model.ActionDropTablePartition: From 25f9b9c38730252f5c9a90832eecf9a39a84e284 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sat, 20 Mar 2021 00:37:18 +0800 Subject: [PATCH 37/61] Optimize code --- ddl/ddl_api.go | 68 +++++++++++++++++++++++--------------------------- 1 file changed, 31 insertions(+), 37 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index fc4a18144dd5d..54357525d64ab 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5183,24 +5183,12 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI } indexInfo := t.Meta().FindIndexByName(indexName.L) - var isPK bool - if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && - // Before we fixed #14243, there might be a general index named `primary` but not a primary key. - (indexInfo == nil || indexInfo.Primary) { - isPK = true - } - if isPK { - // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. - if indexInfo == nil && !t.Meta().PKIsHandle { - return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") - } - if t.Meta().PKIsHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") - } - if t.Meta().IsCommonHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") - } + + isPK, err := checkIsDropPrimaryKey(indexName, indexInfo, t) + if err != nil { + return err } + if indexInfo == nil { err = ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) if ifExists { @@ -5283,27 +5271,10 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt uniqueIndex[indexName.L] = spec.IfExists indexInfo := t.Meta().FindIndexByName(indexName.L) - var isPK bool - if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && - // Before we fixed #14243, there might be a general index named `primary` but not a primary key. - (indexInfo == nil || indexInfo.Primary) { - isPK = true - } - if isPK { - if !config.GetGlobalConfig().AlterPrimaryKey { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when alter-primary-key is false") - } - // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. - if indexInfo == nil && !t.Meta().PKIsHandle { - return ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") - } - if t.Meta().PKIsHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") - } - if t.Meta().IsCommonHandle { - return ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") - } + _, err := checkIsDropPrimaryKey(indexName, indexInfo, t) + if err != nil { + return err } if indexInfo == nil { @@ -5345,6 +5316,29 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt return errors.Trace(err) } +func checkIsDropPrimaryKey(indexName model.CIStr, indexInfo *model.IndexInfo, t table.Table) (bool, error) { + var isPK bool + if indexName.L == strings.ToLower(mysql.PrimaryKeyName) && + // Before we fixed #14243, there might be a general index named `primary` but not a primary key. + (indexInfo == nil || indexInfo.Primary) { + isPK = true + } + if isPK { + // If the table's PKIsHandle is true, we can't find the index from the table. So we check the value of PKIsHandle. + if indexInfo == nil && !t.Meta().PKIsHandle { + return isPK, ErrCantDropFieldOrKey.GenWithStack("Can't DROP 'PRIMARY'; check that column/key exists") + } + if t.Meta().PKIsHandle { + return isPK, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table's pkIsHandle is true") + } + if t.Meta().IsCommonHandle { + return isPK, ErrUnsupportedModifyPrimaryKey.GenWithStack("Unsupported drop primary key when the table is using clustered index") + } + } + + return isPK, nil +} + func isDroppableColumn(tblInfo *model.TableInfo, colName model.CIStr) error { // Check whether there are other columns depend on this column or not. for _, col := range tblInfo.Columns { From ffcd5ba5e6eba80317f68a291a5c427537e1a664 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sat, 20 Mar 2021 14:59:55 +0800 Subject: [PATCH 38/61] Update test cases --- ddl/db_test.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 1c7f8b3ae02bd..33739c9a43cbe 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -6810,8 +6810,9 @@ func (s *testDBSuite5) TestDropIndexes(c *C) { testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) // test drop primary key and index - dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" - idxNames = []string{"primary", "i1"} + createSQL = "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id) nonclustered, unique key i1(c1), key i2(c2));" + dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i2;" + idxNames = []string{"primary", "i2"} testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) testDropDuplicateIndexes(c, s.store) From 5f06638d8789edbac04a8ed716c1798e961d977a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= <32380374+ou-bing@users.noreply.github.com> Date: Tue, 23 Mar 2021 21:04:25 +0800 Subject: [PATCH 39/61] Optimize code Co-authored-by: Lynn --- ddl/index.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 68cfbdfcfdc40..6da6a0669c9f7 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -856,11 +856,11 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. // Double check the index is exists indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { - if !ifExists[i] { - job.State = model.JobStateCancelled - return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + if ifExists[i] { + continue } - continue + job.State = model.JobStateCancelled + return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } // Double check for drop index on auto_increment column. From 4ec2f31000dbc9a4796cff34a0c70d97acbefca5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= <32380374+ou-bing@users.noreply.github.com> Date: Tue, 23 Mar 2021 21:06:55 +0800 Subject: [PATCH 40/61] typo Co-authored-by: Lynn --- 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 33739c9a43cbe..314fa782c1395 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -6619,7 +6619,7 @@ func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dro tk.MustExec("delete from test_drop_indexes") num := 100 - // add some rows + // add some rows for i := 0; i < num; i++ { tk.MustExec("insert into test_drop_indexes values (?, ?, ?)", i, i, i) } From 7361b2c4e044d8e4b33fd991485d1c54db4a2ccd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Tue, 23 Mar 2021 21:10:06 +0800 Subject: [PATCH 41/61] typo --- ddl/db_test.go | 1 - ddl/index.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 33739c9a43cbe..b00e5d6ad37c0 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -6616,7 +6616,6 @@ func testDropIndexes(c *C, store kv.Storage, lease time.Duration, createSQL, dro tk.MustExec("drop table if exists test_drop_indexes") tk.MustExec(createSQL) done := make(chan error, 1) - tk.MustExec("delete from test_drop_indexes") num := 100 // add some rows diff --git a/ddl/index.go b/ddl/index.go index 6da6a0669c9f7..9b8c4434e47b5 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -853,7 +853,7 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. uniqueIndex := make(map[string]bool, len(indexNames)) for i, indexName := range indexNames { - // Double check the index is exists + // Double check the index is exists. indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { if ifExists[i] { From 76006c0355cc7c62f90a292cc89985a9479af65b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 29 Mar 2021 00:02:41 +0800 Subject: [PATCH 42/61] update test case --- ddl/db_test.go | 65 ++++++++++++++++++++++++++++---------------------- 1 file changed, 36 insertions(+), 29 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index b91381374fbd9..0b7b89b4ae20c 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -4487,10 +4487,6 @@ func (s *testDBSuite4) TestIfExists(c *C) { s.mustExec(tk, c, "alter table t2 drop partition if exists p1") c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(1)) tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1507|Error in list of partitions to p1")) - - // DROP INDEXES - s.mustExec(tk, c, "alter table t1 drop index if exists idxes_a,drop index if exists idxes_b") - c.Assert(tk.Se.GetSessionVars().StmtCtx.WarningCount(), Equals, uint16(2)) } func testAddIndexForGeneratedColumn(tk *testkit.TestKit, s *testDBSuite5, c *C) { @@ -6676,28 +6672,6 @@ LOOP: idx := tables.NewIndex(t.Meta().ID, t.Meta(), idx.Meta()) checkDelRangeDone(c, ctx, idx) } - - tk.MustExec("drop table test_drop_indexes") -} - -func testDropDuplicateIndexes(c *C, store kv.Storage) { - tk := testkit.NewTestKitWithInit(c, store) - tk.MustExec("use test_db") - tk.MustExec("drop table if exists test_drop_duplicate_indexes;") - tk.MustExec("create table test_drop_duplicate_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") - - if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index i1, drop index i1;"); true { - c.Assert(err.Error(), Equals, "[ddl:1091]index i1 doesn't exist") - } - - tk.MustExec("alter table test_drop_duplicate_indexes drop index i1, drop index if exists i1;") - tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Note|1091|index i1 doesn't exist")) - - if _, err := tk.Exec("alter table test_drop_duplicate_indexes drop index if exists i2, drop index i2;"); true { - c.Assert(err.Error(), Equals, "[ddl:1091]index i2 doesn't exist") - } - - tk.MustExec("drop table if exists test_drop_duplicate_indexes") } func testCancelDropIndexes(c *C, store kv.Storage, d ddl.DDL) { @@ -6801,6 +6775,41 @@ func testCancelDropIndexes(c *C, store kv.Storage, d ddl.DDL) { tk.MustExec(dropIdxesSQL) } +func testDropIndexesIfExists(c *C, store kv.Storage) { + tk := testkit.NewTestKitWithInit(c, store) + tk.MustExec("use test_db;") + tk.MustExec("drop table if exists test_drop_indexes_if_exists;") + tk.MustExec("create table test_drop_indexes_if_exists (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));") + + // Drop different indexes. + tk.MustGetErrMsg( + "alter table test_drop_indexes_if_exists drop index i1, drop index i3;", + "[ddl:1091]index i3 doesn't exist", + ) + if _, err := tk.Exec("alter table test_drop_indexes_if_exists drop index i1, drop index if exists i3;"); true { + c.Assert(err, IsNil) + } + tk.MustQuery("show warnings").Check( + testutil.RowsWithSep("|", "Note|1091|index i3 doesn't exist"), + ) + + // Verify the impact of deletion order when dropping duplicate indexes. + tk.MustGetErrMsg( + "alter table test_drop_indexes_if_exists drop index i2, drop index i2;", + "[ddl:1091]index i2 doesn't exist", + ) + tk.MustGetErrMsg( + "alter table test_drop_indexes_if_exists drop index if exists i2, drop index i2;", + "[ddl:1091]index i2 doesn't exist", + ) + if _, err := tk.Exec("alter table test_drop_indexes_if_exists drop index i2, drop index if exists i2;"); true { + c.Assert(err, IsNil) + } + tk.MustQuery("show warnings").Check( + testutil.RowsWithSep("|", "Note|1091|index i2 doesn't exist"), + ) +} + func (s *testDBSuite5) TestDropIndexes(c *C) { // drop multiple indexes createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));" @@ -6813,8 +6822,6 @@ func (s *testDBSuite5) TestDropIndexes(c *C) { dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i2;" idxNames = []string{"primary", "i2"} testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) - - testDropDuplicateIndexes(c, s.store) - testCancelDropIndexes(c, s.store, s.dom.DDL()) + testDropIndexesIfExists(c, s.store) } From 36088b19e73290f9e09f454d5e78855afb4ad662 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Tue, 30 Mar 2021 11:48:06 +0800 Subject: [PATCH 43/61] add func: checkNewIndexes --- ddl/index.go | 46 ++++++++++++++++++++++++---------------------- 1 file changed, 24 insertions(+), 22 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 9b8c4434e47b5..47eddda1129d3 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -722,17 +722,7 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde } // Check that drop primary index will not cause invisible implicit primary index. - newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if idx.Name.L != indexInfo.Name.L { - newIndices = append(newIndices, idx) - } - } - newTbl := tblInfo.Clone() - newTbl.Indices = newIndices - err = checkInvisibleIndexOnPK(newTbl) - if err != nil { - job.State = model.JobStateCancelled + if err := checkNewIndexes(tblInfo, []*model.IndexInfo{indexInfo}, job); err != nil { return nil, nil, errors.Trace(err) } @@ -850,14 +840,12 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. } indexInfos := make([]*model.IndexInfo, 0, len(indexNames)) - uniqueIndex := make(map[string]bool, len(indexNames)) - for i, indexName := range indexNames { // Double check the index is exists. indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { if ifExists[i] { - continue + continue } job.State = model.JobStateCancelled return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) @@ -870,25 +858,39 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return nil, nil, autoid.ErrWrongAutoKey } indexInfos = append(indexInfos, indexInfo) - uniqueIndex[indexName.L] = true } // Check that drop primary index will not cause invisible implicit primary index. + if err := checkNewIndexes(tblInfo, indexInfos, job); err != nil { + return nil, nil, errors.Trace(err) + } + + return tblInfo, indexInfos, nil +} + +func checkNewIndexes(tblInfo *model.TableInfo, indexInfos []*model.IndexInfo, job *model.Job) error { newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - for _, idx := range tblInfo.Indices { - if _, ok := uniqueIndex[idx.Name.L]; !ok { - newIndices = append(newIndices, idx) + // tblInfo.FindIndexByName() + for _, oidx := range tblInfo.Indices { + needAppend := true + for _, idx := range indexInfos { + if idx.Name.L == oidx.Name.L { + needAppend = false + break + } + } + if needAppend { + newIndices = append(newIndices, oidx) } } newTbl := tblInfo.Clone() newTbl.Indices = newIndices - err = checkInvisibleIndexOnPK(newTbl) - if err != nil { + if err := checkInvisibleIndexOnPK(newTbl); err != nil { job.State = model.JobStateCancelled - return nil, nil, errors.Trace(err) + return err } - return tblInfo, indexInfos, nil + return nil } func checkDropIndexOnAutoIncrementColumn(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) error { From bcac1fdcf415c72cf4821848fc60b15ee34a31be Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Tue, 30 Mar 2021 13:39:28 +0800 Subject: [PATCH 44/61] Remove rollback mechanism --- ddl/index.go | 12 ++---------- 1 file changed, 2 insertions(+), 10 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 47eddda1129d3..8e87d9792eb54 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -810,16 +810,8 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } - // Finish this job. - if job.IsRollingback() { - job.FinishTableJob(model.JobStateRollbackDone, model.StateNone, ver, tblInfo) - job.Args[0] = indexIDs - // the partition ids were append by convertAddIdxJob2RollbackJob, it is weird, but for the compatibility, - // we should keep appending the partitions in the convertAddIdxJob2RollbackJob. - } else { - job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) - job.Args = append(job.Args, indexIDs, getPartitionIDs(tblInfo)) - } + job.FinishTableJob(model.JobStateDone, model.StateNone, ver, tblInfo) + job.Args = append(job.Args, indexIDs, getPartitionIDs(tblInfo)) default: err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfos[0].State) } From b24c9db4253d5dcc8077e5bf73bca41489382ecc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 5 Apr 2021 20:00:49 +0800 Subject: [PATCH 45/61] add para: ifExists --- ddl/index.go | 16 ++++++++-------- ddl/rollingback.go | 4 ++-- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/ddl/index.go b/ddl/index.go index 8e87d9792eb54..51ff0e19a3f81 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -730,7 +730,7 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde } func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { - tblInfo, indexInfos, err := checkDropIndexes(t, job) + tblInfo, indexInfos, _, err := checkDropIndexes(t, job) if err != nil { return ver, errors.Trace(err) } @@ -818,17 +818,17 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } -func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model.IndexInfo, error) { +func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model.IndexInfo, []bool, error) { schemaID := job.SchemaID tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { - return nil, nil, errors.Trace(err) + return nil, nil, nil, errors.Trace(err) } var indexNames []model.CIStr var ifExists []bool if err = job.DecodeArgs(&indexNames, &ifExists); err != nil { - return nil, nil, errors.Trace(err) + return nil, nil, nil, errors.Trace(err) } indexInfos := make([]*model.IndexInfo, 0, len(indexNames)) @@ -840,24 +840,24 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. continue } job.State = model.JobStateCancelled - return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + return nil, nil, nil, 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, nil, autoid.ErrWrongAutoKey } indexInfos = append(indexInfos, indexInfo) } // Check that drop primary index will not cause invisible implicit primary index. if err := checkNewIndexes(tblInfo, indexInfos, job); err != nil { - return nil, nil, errors.Trace(err) + return nil, nil, nil, errors.Trace(err) } - return tblInfo, indexInfos, nil + return tblInfo, indexInfos, ifExists, nil } func checkNewIndexes(tblInfo *model.TableInfo, indexInfos []*model.IndexInfo, job *model.Job) error { diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 3dc0980fcaabf..41c64be77b321 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -286,7 +286,7 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { } func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) { - tblInfo, indexInfos, err := checkDropIndexes(t, job) + tblInfo, indexInfos, ifExists, err := checkDropIndexes(t, job) if err != nil { return ver, errors.Trace(err) } @@ -313,7 +313,7 @@ func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) for _, indexInfo := range indexInfos { indexNames = append(indexNames, indexInfo.Name) } - job.Args = []interface{}{indexNames} + job.Args = []interface{}{indexNames, ifExists} ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) if err != nil { return ver, errors.Trace(err) From d76d92688353b5829b9adca69bb9bf5f5178c914 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= <32380374+ou-bing@users.noreply.github.com> Date: Thu, 8 Apr 2021 10:08:23 +0800 Subject: [PATCH 46/61] Update ddl/rollingback.go Co-authored-by: Lynn --- ddl/rollingback.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 41c64be77b321..dec979a0f963e 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -296,7 +296,7 @@ func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) switch indexInfo.State { case model.StateWriteOnly, model.StateDeleteOnly, model.StateDeleteReorganization, model.StateNone: // 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. + // Normally won't fetch here, because there is a check when canceling DDL jobs. See function: IsJobRollbackable. job.State = model.JobStateRunning return ver, nil case model.StatePublic: From 424c95627fae020ffe34b718a7825e9e2ba4d8b5 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= <32380374+ou-bing@users.noreply.github.com> Date: Thu, 8 Apr 2021 10:08:46 +0800 Subject: [PATCH 47/61] Update ddl/delete_range.go Co-authored-by: Lynn --- ddl/delete_range.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index d14851c00c9f1..6b888ed2b05b5 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -358,7 +358,7 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error if len(partitionIDs) == 0 { return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) } - for _, pid := range partitionIDs { + for _, pID := range partitionIDs { if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { return errors.Trace(err) } From e7b376f50cb107f4011d4e17e05c5b334abe6660 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 11 Apr 2021 22:34:34 +0800 Subject: [PATCH 48/61] typo --- ddl/delete_range.go | 2 +- ddl/index.go | 7 +++---- 2 files changed, 4 insertions(+), 5 deletions(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 6b888ed2b05b5..142c633a837ec 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -359,7 +359,7 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) } for _, pID := range partitionIDs { - if err := doBatchDeleteIndiceRange(s, job.ID, pid, indexIDs, now); err != nil { + if err := doBatchDeleteIndiceRange(s, job.ID, pID, indexIDs, now); err != nil { return errors.Trace(err) } } diff --git a/ddl/index.go b/ddl/index.go index 51ff0e19a3f81..44f56ebbe6de9 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -722,7 +722,7 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde } // Check that drop primary index will not cause invisible implicit primary index. - if err := checkNewIndexes(tblInfo, []*model.IndexInfo{indexInfo}, job); err != nil { + if err := checkInvisibleIndexesOnPK(tblInfo, []*model.IndexInfo{indexInfo}, job); err != nil { return nil, nil, errors.Trace(err) } @@ -853,16 +853,15 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. } // Check that drop primary index will not cause invisible implicit primary index. - if err := checkNewIndexes(tblInfo, indexInfos, job); err != nil { + if err := checkInvisibleIndexesOnPK(tblInfo, indexInfos, job); err != nil { return nil, nil, nil, errors.Trace(err) } return tblInfo, indexInfos, ifExists, nil } -func checkNewIndexes(tblInfo *model.TableInfo, indexInfos []*model.IndexInfo, job *model.Job) error { +func checkInvisibleIndexesOnPK(tblInfo *model.TableInfo, indexInfos []*model.IndexInfo, job *model.Job) error { newIndices := make([]*model.IndexInfo, 0, len(tblInfo.Indices)) - // tblInfo.FindIndexByName() for _, oidx := range tblInfo.Indices { needAppend := true for _, idx := range indexInfos { From 70a8b0ff3be6211005eee317d47e030e670ca189 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 11 Apr 2021 23:31:31 +0800 Subject: [PATCH 49/61] remove func: getSchemaAndTable --- ddl/ddl_api.go | 15 +-------------- 1 file changed, 1 insertion(+), 14 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 54357525d64ab..6607b1b4378f4 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5228,21 +5228,8 @@ func (d *ddl) DropIndex(ctx sessionctx.Context, ti ast.Ident, indexName model.CI return errors.Trace(err) } -func (d *ddl) getSchemaAndTable(ti ast.Ident) (*model.DBInfo, table.Table, error) { - is := d.infoHandle.Get() - schema, ok := is.SchemaByName(ti.Schema) - if !ok { - return nil, nil, errors.Trace(infoschema.ErrDatabaseNotExists) - } - t, err := is.TableByName(ti.Schema, ti.Name) - if err != nil { - return nil, nil, errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ti.Schema, ti.Name)) - } - return schema, t, nil -} - func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.AlterTableSpec) error { - schema, t, err := d.getSchemaAndTable(ti) + schema, t, err := d.getSchemaAndTableByIdent(ctx, ti) if err != nil { return err } From 760bfdcd90de0bba6c91d9df00a39f9e69e6358a Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 21 Apr 2021 11:10:51 +0800 Subject: [PATCH 50/61] add case to `IsJobRollbackable` --- util/admin/admin.go | 2 +- util/admin/admin_test.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/util/admin/admin.go b/util/admin/admin.go index 20217a53c1b6d..31b92dd880442 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -91,7 +91,7 @@ func GetDDLInfo(txn kv.Transaction) (*DDLInfo, error) { // IsJobRollbackable checks whether the job can be rollback. func IsJobRollbackable(job *model.Job) bool { switch job.Type { - case model.ActionDropIndex, model.ActionDropPrimaryKey: + case model.ActionDropIndex, model.ActionDropPrimaryKey, model.ActionDropIndexes: // We can't cancel if index current state is in StateDeleteOnly or StateDeleteReorganization or StateWriteOnly, otherwise there will be an inconsistent issue between record and index. // In WriteOnly state, we can rollback for normal index but can't rollback for expression index(need to drop hidden column). Since we can't // know the type of index here, we consider all indices except primary index as non-rollbackable. diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index fa7f4316173b4..fe7be32419521 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -370,6 +370,7 @@ func (s *testSuite) TestIsJobRollbackable(c *C) { {model.ActionDropSchema, model.StateDeleteOnly, false}, {model.ActionDropColumn, model.StateDeleteOnly, false}, {model.ActionDropColumns, model.StateDeleteOnly, false}, + {model.ActionDropIndexes, model.StateDeleteOnly, false}, } job := &model.Job{} for _, ca := range cases { From ec8ee317581c6ad883c39e1ecf91b6096821aa48 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 21 Apr 2021 13:23:55 +0800 Subject: [PATCH 51/61] add test case --- util/admin/admin_test.go | 24 +++++++++++++++++++----- 1 file changed, 19 insertions(+), 5 deletions(-) diff --git a/util/admin/admin_test.go b/util/admin/admin_test.go index fe7be32419521..17242bdd65761 100644 --- a/util/admin/admin_test.go +++ b/util/admin/admin_test.go @@ -253,16 +253,30 @@ func (s *testSuite) TestCancelJobs(c *C) { c.Assert(errs[0].Error(), Matches, "*This job:100 is finished, so can't be cancelled") // test can't cancelable job. - job.Type = model.ActionDropIndex - job.SchemaState = model.StateWriteOnly - job.State = model.JobStateRunning - job.ID = 101 + job = &model.Job{ + ID: 101, + SchemaID: 1, + Type: model.ActionDropIndex, + SchemaState: model.StateWriteOnly, + State: model.JobStateRunning, + } + err = t.EnQueueDDLJob(job) + c.Assert(err, IsNil) + job = &model.Job{ + ID: 102, + SchemaID: 1, + Type: model.ActionDropIndexes, + SchemaState: model.StateWriteOnly, + State: model.JobStateRunning, + } err = t.EnQueueDDLJob(job) c.Assert(err, IsNil) - errs, err = CancelJobs(txn, []int64{101}) + errs, err = CancelJobs(txn, []int64{101, 102}) c.Assert(err, IsNil) c.Assert(errs[0], NotNil) + c.Assert(errs[1], NotNil) c.Assert(errs[0].Error(), Matches, "*This job:101 is almost finished, can't be cancelled now") + c.Assert(errs[1].Error(), Matches, "*This job:102 is almost finished, can't be cancelled now") // When both types of jobs exist in the DDL queue, // we first cancel the job with a larger ID. From 62af0e82d24c5f8ffd6a65061390ee3c33b48bb2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sat, 24 Apr 2021 22:39:03 +0800 Subject: [PATCH 52/61] add test case --- ddl/db_test.go | 20 +++++++++++++++++++- 1 file changed, 19 insertions(+), 1 deletion(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 9d30b8d96832d..e0f2c95aa997f 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -6905,6 +6905,23 @@ func testDropIndexesIfExists(c *C, store kv.Storage) { ) } +func testDropIndexesFromPartitionedTable(c *C, store kv.Storage) { + tk := testkit.NewTestKitWithInit(c, store) + tk.MustExec("use test_db;") + tk.MustExec("drop table if exists test_drop_indexes_from_partitioned_table;") + tk.MustExec(` + create table test_drop_indexes_from_partitioned_table (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2)) + partition by range(id) (partition p0 values less than (6), partition p1 values less than maxvalue); + `) + for i := 0; i < 20; i++ { + tk.MustExec("insert into test_drop_indexes_from_partitioned_table values (?, ?, ?)", i, i, i) + } + if _, err := tk.Exec("alter table test_drop_indexes_from_partitioned_table drop index i1, drop index if exists i2;"); true { + c.Assert(err, IsNil) + } + +} + func (s *testDBSuite5) TestDropIndexes(c *C) { // drop multiple indexes createSQL := "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id), key i1(c1), key i2(c2));" @@ -6919,4 +6936,5 @@ func (s *testDBSuite5) TestDropIndexes(c *C) { testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) testCancelDropIndexes(c, s.store, s.dom.DDL()) testDropIndexesIfExists(c, s.store) -} \ No newline at end of file + testDropIndexesFromPartitionedTable(c, s.store) +} From 7483cdc402849d17c42d4613f6add4ce13c3cd43 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Tue, 27 Apr 2021 15:57:00 +0800 Subject: [PATCH 53/61] add test case --- ddl/ddl_worker_test.go | 46 ++++++++++++++++++++++++++++++++++++++++++ ddl/table_test.go | 18 +++++++++++++++++ 2 files changed, 64 insertions(+) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index 6e745820b04b9..a787db55242b6 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -649,6 +649,13 @@ func buildCancelJobTests(firstID int64) []testCancelJob { {act: model.ActionModifyColumn, jobIDs: []int64{firstID + 67}, cancelRetErrs: noErrs, cancelState: model.StateWriteOnly}, {act: model.ActionModifyColumn, jobIDs: []int64{firstID + 68}, cancelRetErrs: noErrs, cancelState: model.StateWriteReorganization}, {act: model.ActionModifyColumn, jobIDs: []int64{firstID + 69}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob}, cancelState: model.StatePublic}, + + // for drop indexes + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 70}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 70)}, cancelState: model.StateWriteOnly}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 71}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 71)}, cancelState: model.StateDeleteOnly}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 72}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 72)}, cancelState: model.StateWriteReorganization}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 73}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 73)}, cancelState: model.StateNone}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 74}, cancelRetErrs: noErrs, cancelState: model.StatePublic}, } return tests @@ -1279,6 +1286,39 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { c.Assert(baseTable.Meta().Columns[0].FieldType.Tp, Equals, mysql.TypeTiny) c.Assert(baseTable.Meta().Columns[0].FieldType.Flag&mysql.NotNullFlag, Equals, uint(1)) c.Assert(failpoint.Disable("github.com/pingcap/tidb/ddl/skipMockContextDoExec"), IsNil) + + // for drop indexes + updateTest(&tests[54]) + ifExists := make([]bool, 2) + idxNames := []model.CIStr{model.NewCIStr("i1"), model.NewCIStr("i2")} + dropIndexesArgs := []interface{}{idxNames, ifExists} + tableInfo := createTestTableForDropIndexes(c, ctx, d, dbInfo, "test-drop-indexes", 10) + doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + + updateTest(&tests[55]) + idxNames = []model.CIStr{model.NewCIStr("i3"), model.NewCIStr("i4")} + dropIndexesArgs = []interface{}{idxNames, ifExists} + doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + + updateTest(&tests[56]) + idxNames = []model.CIStr{model.NewCIStr("i5"), model.NewCIStr("i6")} + dropIndexesArgs = []interface{}{idxNames, ifExists} + doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + + updateTest(&tests[57]) + idxNames = []model.CIStr{model.NewCIStr("i7"), model.NewCIStr("i8")} + dropIndexesArgs = []interface{}{idxNames, ifExists} + doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) + + updateTest(&tests[58]) + idxNames = []model.CIStr{model.NewCIStr("i9"), model.NewCIStr("i10")} + dropIndexesArgs = []interface{}{idxNames, ifExists} + doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) + s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, false) } func (s *testDDLSuite) TestIgnorableSpec(c *C) { @@ -1625,3 +1665,9 @@ func (s *testDDLSuite) TestDDLPackageExecuteSQL(c *C) { se := sess.(sqlexec.SQLExecutor) _, _ = se.Execute(context.Background(), "create table t(a int);") } + +func (s *testDDLSerialSuite) checkDropIndexes(c *C, d *ddl, schemaID int64, tableID int64, idxNames []model.CIStr, success bool) { + for _, idxName := range idxNames { + checkIdxExist(c, d, schemaID, tableID, idxName.O, !success) + } +} diff --git a/ddl/table_test.go b/ddl/table_test.go index 5760fc2b152b5..5610313353890 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -412,3 +412,21 @@ func (s *testTableSuite) TestTable(c *C) { testCheckJobDone(c, d, job, true) checkTableLockedTest(c, d, dbInfo1, tblInfo, d.GetID(), ctx.GetSessionVars().ConnectionID, model.TableLockWrite) } + +// for drop indexes +func createTestTableForDropIndexes(c *C, ctx sessionctx.Context, d *ddl, dbInfo *model.DBInfo, name string, num int) *model.TableInfo { + tableInfo := testTableInfo(c, d, name, num) + var idxs []*model.IndexInfo + for i := 0; i < num; i++ { + idxName := model.NewCIStr(fmt.Sprintf("i%d", i+1)) + idx := &model.IndexInfo{ + Name: idxName, + State: model.StatePublic, + Columns: []*model.IndexColumn{{Name: model.NewCIStr(fmt.Sprintf("c%d", i+1))}}, + } + idxs = append(idxs, idx) + } + tableInfo.Indices = idxs + testCreateTable(c, ctx, d, dbInfo, tableInfo) + return tableInfo +} From 69fd5bb1bf70c6bd733bd45c2effd429c3f3ef6e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 10 May 2021 21:34:23 +0800 Subject: [PATCH 54/61] add test case --- ddl/ddl_worker_test.go | 22 ++++------------------ 1 file changed, 4 insertions(+), 18 deletions(-) diff --git a/ddl/ddl_worker_test.go b/ddl/ddl_worker_test.go index a787db55242b6..76a0f27ddef6e 100644 --- a/ddl/ddl_worker_test.go +++ b/ddl/ddl_worker_test.go @@ -651,11 +651,9 @@ func buildCancelJobTests(firstID int64) []testCancelJob { {act: model.ActionModifyColumn, jobIDs: []int64{firstID + 69}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob}, cancelState: model.StatePublic}, // for drop indexes - {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 70}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 70)}, cancelState: model.StateWriteOnly}, - {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 71}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 71)}, cancelState: model.StateDeleteOnly}, - {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 72}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 72)}, cancelState: model.StateWriteReorganization}, - {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 73}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 73)}, cancelState: model.StateNone}, - {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 74}, cancelRetErrs: noErrs, cancelState: model.StatePublic}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 72}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 72)}, cancelState: model.StateWriteOnly}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 73}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 73)}, cancelState: model.StateDeleteOnly}, + {act: model.ActionDropIndexes, jobIDs: []int64{firstID + 74}, cancelRetErrs: []error{admin.ErrCannotCancelDDLJob.GenWithStackByArgs(firstID + 74)}, cancelState: model.StateWriteReorganization}, } return tests @@ -1292,7 +1290,7 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { ifExists := make([]bool, 2) idxNames := []model.CIStr{model.NewCIStr("i1"), model.NewCIStr("i2")} dropIndexesArgs := []interface{}{idxNames, ifExists} - tableInfo := createTestTableForDropIndexes(c, ctx, d, dbInfo, "test-drop-indexes", 10) + tableInfo := createTestTableForDropIndexes(c, ctx, d, dbInfo, "test-drop-indexes", 6) doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) @@ -1307,18 +1305,6 @@ func (s *testDDLSerialSuite) TestCancelJob(c *C) { dropIndexesArgs = []interface{}{idxNames, ifExists} doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) - - updateTest(&tests[57]) - idxNames = []model.CIStr{model.NewCIStr("i7"), model.NewCIStr("i8")} - dropIndexesArgs = []interface{}{idxNames, ifExists} - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) - s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, true) - - updateTest(&tests[58]) - idxNames = []model.CIStr{model.NewCIStr("i9"), model.NewCIStr("i10")} - dropIndexesArgs = []interface{}{idxNames, ifExists} - doDDLJobSuccess(ctx, d, c, dbInfo.ID, tableInfo.ID, test.act, dropIndexesArgs) - s.checkDropIndexes(c, d, dbInfo.ID, tableInfo.ID, idxNames, false) } func (s *testDDLSuite) TestIgnorableSpec(c *C) { From 6952ac76791746b51dd553ab675f7c69ef43ebd3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 28 Jun 2021 09:20:57 +0800 Subject: [PATCH 55/61] Transform the judgment of "if exists" --- ddl/ddl_api.go | 49 +++++++++++++-------------------------------- ddl/delete_range.go | 4 +--- ddl/index.go | 43 ++++++++++++++++++++++++++++++++------- ddl/rollingback.go | 12 +++++------ 4 files changed, 57 insertions(+), 51 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index c0f86ec9e24ac..601d853d2969a 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5405,10 +5405,8 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt return err } - uniqueIndex := make(map[string]bool, len(specs)) indexNames := make([]model.CIStr, 0, len(specs)) ifExists := make([]bool, 0, len(specs)) - for _, spec := range specs { var indexName model.CIStr if spec.Tp == ast.AlterTableDropPrimaryKey { @@ -5417,46 +5415,21 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt indexName = model.NewCIStr(spec.Name) } - _, ok := uniqueIndex[indexName.L] - if ok { - err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) - if spec.IfExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - continue - } - return err - } - uniqueIndex[indexName.L] = spec.IfExists - indexInfo := t.Meta().FindIndexByName(indexName.L) - - _, err := checkIsDropPrimaryKey(indexName, indexInfo, t) - if err != nil { - return err - } - - if indexInfo == nil { - err := ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) - if spec.IfExists { - ctx.GetSessionVars().StmtCtx.AppendNote(err) - continue + if indexInfo != nil { + _, err := checkIsDropPrimaryKey(indexName, indexInfo, t) + if err != nil { + return err + } + if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { + return errors.Trace(err) } - return err - } - - // Check for drop index on auto_increment column. - if err := checkDropIndexOnAutoIncrementColumn(t.Meta(), indexInfo); err != nil { - return errors.Trace(err) } indexNames = append(indexNames, indexName) ifExists = append(ifExists, spec.IfExists) } - if len(indexNames) == 0 { - return nil - } - job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, @@ -5468,7 +5441,13 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt err = d.doDDLJob(ctx, job) if err != nil { - return errors.Trace(err) + if err, ok := err.(errors.); ok { + for _, indexName := range err.IndexNames { + ctx.GetSessionVars().StmtCtx.AppendNote(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName)) + } + return nil + } + } err = d.callHookOnChanged(err) return errors.Trace(err) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 71c08574563fd..d70b6f515ff2b 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -343,11 +343,9 @@ func insertJobIntoDeleteRangeTable(ctx sessionctx.Context, job *model.Job) error return doInsert(s, job.ID, indexID, startKey, endKey, now) } case model.ActionDropIndexes: - var indexNames []model.CIStr - var ifExists []bool var indexIDs []int64 var partitionIDs []int64 - if err := job.DecodeArgs(&indexNames, &ifExists, &indexIDs, &partitionIDs); err != nil { + if err := job.DecodeArgs(&[]model.CIStr{}, &[]bool{}, &indexIDs, &partitionIDs); err != nil { return errors.Trace(err) } // Remove data in TiKV. diff --git a/ddl/index.go b/ddl/index.go index 173339ecb232f..e2fbddb4be383 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -717,14 +717,33 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde return tblInfo, indexInfo, nil } +type dropIndexesWarns struct { + IndexNames []model.CIStr +} + +func (warns *dropIndexesWarns) Error() string { + return "" +} + func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { - tblInfo, indexInfos, _, err := checkDropIndexes(t, job) + tblInfo, indexNames, ifExists, err := getSchemaInfos(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + indexInfos, nonExsitentIndexNames, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) if err != nil { return ver, errors.Trace(err) } + warns := &dropIndexesWarns{} + if len(indexInfos) == 0 { job.State = model.JobStateCancelled + if len(nonExsitentIndexNames) != 0 { + warns.IndexNames = nonExsitentIndexNames + return ver, warns + } return ver, nil } @@ -803,10 +822,14 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { default: err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfos[0].State) } + + if err == nil && len(nonExsitentIndexNames) != 0 { + return ver, warns + } return ver, errors.Trace(err) } -func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model.IndexInfo, []bool, error) { +func getSchemaInfos(t *meta.Meta, job *model.Job) (*model.TableInfo, []model.CIStr, []bool, error) { schemaID := job.SchemaID tblInfo, err := getTableInfoAndCancelFaultJob(t, job, schemaID) if err != nil { @@ -819,33 +842,39 @@ func checkDropIndexes(t *meta.Meta, job *model.Job) (*model.TableInfo, []*model. return nil, nil, nil, errors.Trace(err) } + return tblInfo, indexNames, ifExists, nil +} + +func checkDropIndexes(tblInfo *model.TableInfo, job *model.Job, indexNames []model.CIStr, ifExists []bool) ([]*model.IndexInfo, []model.CIStr, error) { + nonExistentIndexeNames := make([]model.CIStr, 0, len(indexNames)) indexInfos := make([]*model.IndexInfo, 0, len(indexNames)) for i, indexName := range indexNames { // Double check the index is exists. indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { if ifExists[i] { + nonExistentIndexeNames = append(nonExistentIndexeNames, indexName) continue } job.State = model.JobStateCancelled - return nil, nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } // Double check for drop index on auto_increment column. - err = checkDropIndexOnAutoIncrementColumn(tblInfo, indexInfo) + err := checkDropIndexOnAutoIncrementColumn(tblInfo, indexInfo) if err != nil { job.State = model.JobStateCancelled - return nil, nil, nil, autoid.ErrWrongAutoKey + return nil, nil, autoid.ErrWrongAutoKey } indexInfos = append(indexInfos, indexInfo) } // Check that drop primary index will not cause invisible implicit primary index. if err := checkInvisibleIndexesOnPK(tblInfo, indexInfos, job); err != nil { - return nil, nil, nil, errors.Trace(err) + return nil, nil, errors.Trace(err) } - return tblInfo, indexInfos, ifExists, nil + return indexInfos, nonExistentIndexeNames, nil } func checkInvisibleIndexesOnPK(tblInfo *model.TableInfo, indexInfos []*model.IndexInfo, job *model.Job) error { diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 8d69dae0e981b..791b2ad90fd02 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -288,7 +288,12 @@ func rollingbackDropIndex(t *meta.Meta, job *model.Job) (ver int64, err error) { } func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) { - tblInfo, indexInfos, ifExists, err := checkDropIndexes(t, job) + tblInfo, indexNames, ifExists, err := getSchemaInfos(t, job) + if err != nil { + return ver, errors.Trace(err) + } + + indexInfos, _, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) if err != nil { return ver, errors.Trace(err) } @@ -311,11 +316,6 @@ func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) } job.SchemaState = indexInfo.State - var indexNames []model.CIStr - for _, indexInfo := range indexInfos { - indexNames = append(indexNames, indexInfo.Name) - } - job.Args = []interface{}{indexNames, ifExists} ver, err = updateVersionAndTableInfo(t, job, tblInfo, originalState != indexInfo.State) if err != nil { return ver, errors.Trace(err) From 73f5c9e9c7227baf0bc23f83b1c7ce9dbac7835b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Sun, 11 Jul 2021 20:18:34 +0800 Subject: [PATCH 56/61] An error is thrown when deleting an invalid index. --- ddl/ddl.go | 16 ++++++++++++++-- ddl/ddl_api.go | 9 --------- ddl/index.go | 15 +++++++-------- 3 files changed, 21 insertions(+), 19 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 9eb05b86741ed..502536169bdf9 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -598,6 +598,13 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error { } } } + + if historyJob.MultiSchemaInfo != nil && len(historyJob.MultiSchemaInfo.Warnings) != 0 { + for _, warning := range historyJob.MultiSchemaInfo.Warnings { + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } + } + logutil.BgLogger().Info("[ddl] DDL job is finished", zap.Int64("jobID", jobID)) return nil } @@ -606,8 +613,13 @@ func (d *ddl) doDDLJob(ctx sessionctx.Context, job *model.Job) error { logutil.BgLogger().Info("[ddl] DDL job is failed", zap.Int64("jobID", jobID)) return errors.Trace(historyJob.Error) } - // Only for JobStateCancelled job which is adding columns or drop columns. - if historyJob.IsCancelled() && (historyJob.Type == model.ActionAddColumns || historyJob.Type == model.ActionDropColumns) { + // Only for JobStateCancelled job which is adding columns or drop columns or drop indexes. + if historyJob.IsCancelled() && (historyJob.Type == model.ActionAddColumns || historyJob.Type == model.ActionDropColumns || historyJob.Type == model.ActionDropIndexes) { + if historyJob.MultiSchemaInfo != nil && len(historyJob.MultiSchemaInfo.Warnings) != 0 { + for _, warning := range historyJob.MultiSchemaInfo.Warnings { + ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + } + } logutil.BgLogger().Info("[ddl] DDL job is cancelled", zap.Int64("jobID", jobID)) return nil } diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 601d853d2969a..efd76780b9d6f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -5440,15 +5440,6 @@ func (d *ddl) DropIndexes(ctx sessionctx.Context, ti ast.Ident, specs []*ast.Alt } err = d.doDDLJob(ctx, job) - if err != nil { - if err, ok := err.(errors.); ok { - for _, indexName := range err.IndexNames { - ctx.GetSessionVars().StmtCtx.AppendNote(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName)) - } - return nil - } - - } err = d.callHookOnChanged(err) return errors.Trace(err) } diff --git a/ddl/index.go b/ddl/index.go index e2fbddb4be383..b9198dbc92baf 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -736,14 +736,16 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } - warns := &dropIndexesWarns{} + if len(nonExsitentIndexNames) != 0 { + var warnings []*errors.Error + for _, indexName := range nonExsitentIndexNames { + warnings = append(warnings, toTError(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName))) + } + job.MultiSchemaInfo = &model.MultiSchemaInfo{Warnings: warnings} + } if len(indexInfos) == 0 { job.State = model.JobStateCancelled - if len(nonExsitentIndexNames) != 0 { - warns.IndexNames = nonExsitentIndexNames - return ver, warns - } return ver, nil } @@ -823,9 +825,6 @@ func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { err = ErrInvalidDDLState.GenWithStackByArgs("index", indexInfos[0].State) } - if err == nil && len(nonExsitentIndexNames) != 0 { - return ver, warns - } return ver, errors.Trace(err) } From ee8c776c61d7c01970daac7bb3da4e23f3dcd943 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Mon, 23 Aug 2021 17:58:07 +0800 Subject: [PATCH 57/61] update code --- ddl/delete_range.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ddl/delete_range.go b/ddl/delete_range.go index 73272e161e26c..6430a8f2a3219 100644 --- a/ddl/delete_range.go +++ b/ddl/delete_range.go @@ -357,10 +357,10 @@ func insertJobIntoDeleteRangeTable(ctx context.Context, sctx sessionctx.Context, return nil } if len(partitionIDs) == 0 { - return doBatchDeleteIndiceRange(s, job.ID, job.TableID, indexIDs, now) + return doBatchDeleteIndiceRange(ctx, s, job.ID, job.TableID, indexIDs, now) } for _, pID := range partitionIDs { - if err := doBatchDeleteIndiceRange(s, job.ID, pID, indexIDs, now); err != nil { + if err := doBatchDeleteIndiceRange(ctx, s, job.ID, pID, indexIDs, now); err != nil { return errors.Trace(err) } } From 27f5db4f41caf10b2cc76155dfcfa16b8c8f16e8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 25 Aug 2021 22:29:48 +0800 Subject: [PATCH 58/61] update code --- ddl/db_test.go | 4 ++-- ddl/index.go | 48 +++++++++++++++++++++------------------------- ddl/rollingback.go | 2 +- 3 files changed, 25 insertions(+), 29 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 4b0dd3b2e2c0d..07c8936d190e5 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -7480,7 +7480,7 @@ func testDropIndexesIfExists(c *C, store kv.Storage) { c.Assert(err, IsNil) } tk.MustQuery("show warnings").Check( - testutil.RowsWithSep("|", "Note|1091|index i3 doesn't exist"), + testutil.RowsWithSep("|", "Warning|1091|index i3 doesn't exist"), ) // Verify the impact of deletion order when dropping duplicate indexes. @@ -7496,7 +7496,7 @@ func testDropIndexesIfExists(c *C, store kv.Storage) { c.Assert(err, IsNil) } tk.MustQuery("show warnings").Check( - testutil.RowsWithSep("|", "Note|1091|index i2 doesn't exist"), + testutil.RowsWithSep("|", "Warning|1091|index i2 doesn't exist"), ) } diff --git a/ddl/index.go b/ddl/index.go index 77a07fb7720ce..453d7bfe66773 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -727,33 +727,17 @@ func checkDropIndex(t *meta.Meta, job *model.Job) (*model.TableInfo, *model.Inde return tblInfo, indexInfo, nil } -type dropIndexesWarns struct { - IndexNames []model.CIStr -} - -func (warns *dropIndexesWarns) Error() string { - return "" -} - func onDropIndexes(t *meta.Meta, job *model.Job) (ver int64, _ error) { tblInfo, indexNames, ifExists, err := getSchemaInfos(t, job) if err != nil { return ver, errors.Trace(err) } - indexInfos, nonExsitentIndexNames, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) + indexInfos, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) if err != nil { return ver, errors.Trace(err) } - if len(nonExsitentIndexNames) != 0 { - var warnings []*errors.Error - for _, indexName := range nonExsitentIndexNames { - warnings = append(warnings, toTError(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName))) - } - job.MultiSchemaInfo = &model.MultiSchemaInfo{Warnings: warnings} - } - if len(indexInfos) == 0 { job.State = model.JobStateCancelled return ver, nil @@ -854,36 +838,48 @@ func getSchemaInfos(t *meta.Meta, job *model.Job) (*model.TableInfo, []model.CIS return tblInfo, indexNames, ifExists, nil } -func checkDropIndexes(tblInfo *model.TableInfo, job *model.Job, indexNames []model.CIStr, ifExists []bool) ([]*model.IndexInfo, []model.CIStr, error) { - nonExistentIndexeNames := make([]model.CIStr, 0, len(indexNames)) +func checkDropIndexes(tblInfo *model.TableInfo, job *model.Job, indexNames []model.CIStr, ifExists []bool) ([]*model.IndexInfo, error) { + var warnings []*errors.Error indexInfos := make([]*model.IndexInfo, 0, len(indexNames)) + UniqueIndexNames := make(map[model.CIStr]bool, len(indexNames)) for i, indexName := range indexNames { // Double check the index is exists. indexInfo := tblInfo.FindIndexByName(indexName.L) if indexInfo == nil { if ifExists[i] { - nonExistentIndexeNames = append(nonExistentIndexeNames, indexName) + warnings = append(warnings, toTError(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName))) continue } job.State = model.JobStateCancelled - return nil, nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + return nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } // Double check for drop index on auto_increment column. - err := checkDropIndexOnAutoIncrementColumn(tblInfo, indexInfo) - if err != nil { + if err := checkDropIndexOnAutoIncrementColumn(tblInfo, indexInfo); err != nil { job.State = model.JobStateCancelled - return nil, nil, autoid.ErrWrongAutoKey + return nil, autoid.ErrWrongAutoKey } + + // Check for dropping duplicate indexes. + if UniqueIndexNames[indexName] { + if !ifExists[i] { + return nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) + } + warnings = append(warnings, toTError(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName))) + } + UniqueIndexNames[indexName] = true + indexInfos = append(indexInfos, indexInfo) } // Check that drop primary index will not cause invisible implicit primary index. if err := checkInvisibleIndexesOnPK(tblInfo, indexInfos, job); err != nil { - return nil, nil, errors.Trace(err) + return nil, errors.Trace(err) } - return indexInfos, nonExistentIndexeNames, nil + job.MultiSchemaInfo = &model.MultiSchemaInfo{Warnings: warnings} + + return indexInfos, nil } func checkInvisibleIndexesOnPK(tblInfo *model.TableInfo, indexInfos []*model.IndexInfo, job *model.Job) error { diff --git a/ddl/rollingback.go b/ddl/rollingback.go index ef691456bdfb4..fbc48f4f69e5f 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -292,7 +292,7 @@ func rollingbackDropIndexes(t *meta.Meta, job *model.Job) (ver int64, err error) return ver, errors.Trace(err) } - indexInfos, _, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) + indexInfos, err := checkDropIndexes(tblInfo, job, indexNames, ifExists) if err != nil { return ver, errors.Trace(err) } From 6556377518ab0c74d197e65860ff7713cdb3c34b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 25 Aug 2021 23:04:25 +0800 Subject: [PATCH 59/61] update code --- ddl/db_test.go | 4 ++-- ddl/index.go | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 07c8936d190e5..813ab55c0c811 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -7479,7 +7479,7 @@ func testDropIndexesIfExists(c *C, store kv.Storage) { if _, err := tk.Exec("alter table test_drop_indexes_if_exists drop index i1, drop index if exists i3;"); true { c.Assert(err, IsNil) } - tk.MustQuery("show warnings").Check( + tk.MustQuery("show warnings;").Check( testutil.RowsWithSep("|", "Warning|1091|index i3 doesn't exist"), ) @@ -7495,7 +7495,7 @@ func testDropIndexesIfExists(c *C, store kv.Storage) { if _, err := tk.Exec("alter table test_drop_indexes_if_exists drop index i2, drop index if exists i2;"); true { c.Assert(err, IsNil) } - tk.MustQuery("show warnings").Check( + tk.MustQuery("show warnings;").Check( testutil.RowsWithSep("|", "Warning|1091|index i2 doesn't exist"), ) } diff --git a/ddl/index.go b/ddl/index.go index 453d7bfe66773..8585487af10a0 100644 --- a/ddl/index.go +++ b/ddl/index.go @@ -863,6 +863,7 @@ func checkDropIndexes(tblInfo *model.TableInfo, job *model.Job, indexNames []mod // Check for dropping duplicate indexes. if UniqueIndexNames[indexName] { if !ifExists[i] { + job.State = model.JobStateCancelled return nil, ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName) } warnings = append(warnings, toTError(ErrCantDropFieldOrKey.GenWithStack("index %s doesn't exist", indexName))) From c3107e81877012b5a128032068819159fff21fe1 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 1 Sep 2021 23:25:37 +0800 Subject: [PATCH 60/61] update func: isSameTypeMultiSpecs --- ddl/ddl_api.go | 22 ++++++---------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 519e32cada2d4..1d028cbb83207 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2410,16 +2410,11 @@ func resolveAlterTableSpec(ctx sessionctx.Context, specs []*ast.AlterTableSpec) func isSameTypeMultiSpecs(specs []*ast.AlterTableSpec) bool { specType := specs[0].Tp for _, spec := range specs { - if spec.Tp != specType { - return false + // We think AlterTableDropPrimaryKey and AlterTableDropIndex are the same types. + if spec.Tp == ast.AlterTableDropPrimaryKey || spec.Tp == ast.AlterTableDropIndex { + continue } - } - return true -} - -func isDropIndexes(specs []*ast.AlterTableSpec) bool { - for _, spec := range specs { - if spec.Tp != ast.AlterTableDropPrimaryKey && spec.Tp != ast.AlterTableDropIndex { + if spec.Tp != specType { return false } } @@ -2442,19 +2437,14 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, ident ast return errRunMultiSchemaChanges } - if isDropIndexes(validSpecs) { - if err = d.DropIndexes(sctx, ident, validSpecs); err != nil { - return errors.Trace(err) - } - return nil - } - if isSameTypeMultiSpecs(validSpecs) { switch validSpecs[0].Tp { case ast.AlterTableAddColumns: err = d.AddColumns(sctx, ident, validSpecs) case ast.AlterTableDropColumn: err = d.DropColumns(sctx, ident, validSpecs) + case ast.AlterTableDropPrimaryKey, ast.AlterTableDropIndex: + err = d.DropIndexes(sctx, ident, validSpecs) default: return errRunMultiSchemaChanges } From 0c1bcd17c02b2c1d009f1341362832170dfdb1c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E6=AC=A7=E5=86=B0?= Date: Wed, 1 Sep 2021 23:43:33 +0800 Subject: [PATCH 61/61] add test case --- ddl/db_test.go | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/ddl/db_test.go b/ddl/db_test.go index 7dd237090b9bf..60d379bc3715a 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -7527,14 +7527,19 @@ func (s *testDBSuite5) TestDropIndexes(c *C) { idxNames := []string{"i1", "i2"} testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) - // test drop primary key and index createSQL = "create table test_drop_indexes (id int, c1 int, c2 int, primary key(id) nonclustered, unique key i1(c1), key i2(c2));" - dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i2;" - idxNames = []string{"primary", "i2"} + dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1;" + idxNames = []string{"primary", "i1"} testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) - testCancelDropIndexes(c, s.store, s.dom.DDL()) + + createSQL = "create table test_drop_indexes (uuid varchar(32), c1 int, c2 int, primary key(uuid), unique key i1(c1), key i2(c2));" + dropIdxSQL = "alter table test_drop_indexes drop primary key, drop index i1, drop index i2;" + idxNames = []string{"primary", "i1", "i2"} + testDropIndexes(c, s.store, s.lease, createSQL, dropIdxSQL, idxNames) + testDropIndexesIfExists(c, s.store) testDropIndexesFromPartitionedTable(c, s.store) + testCancelDropIndexes(c, s.store, s.dom.DDL()) } // Close issue #24580.