Skip to content

Commit

Permalink
ddl: support rename index and columns for multi-schema change (#36148)
Browse files Browse the repository at this point in the history
ref #14766
  • Loading branch information
tangenta authored Jul 13, 2022
1 parent f581ec3 commit 12c1721
Show file tree
Hide file tree
Showing 4 changed files with 301 additions and 2 deletions.
13 changes: 13 additions & 0 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -1349,6 +1349,12 @@ func (w *worker) doModifyColumn(
}
}

if job.MultiSchemaInfo != nil && job.MultiSchemaInfo.Revertible {
job.MarkNonRevertible()
// Store the mark and enter the next DDL handling loop.
return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, false)
}

if err := adjustTableInfoAfterModifyColumn(tblInfo, newCol, oldCol, pos); err != nil {
job.State = model.JobStateRollingback
return ver, errors.Trace(err)
Expand Down Expand Up @@ -1505,6 +1511,13 @@ func updateColumnDefaultValue(d *ddlCtx, t *meta.Meta, job *model.Job, newCol *m
if err != nil {
return ver, errors.Trace(err)
}

if job.MultiSchemaInfo != nil && job.MultiSchemaInfo.Revertible {
job.MarkNonRevertible()
// Store the mark and enter the next DDL handling loop.
return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, false)
}

oldCol := model.FindColumnInfo(tblInfo.Columns, oldColName.L)
if oldCol == nil || oldCol.State != model.StatePublic {
job.State = model.JobStateCancelled
Expand Down
20 changes: 18 additions & 2 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,8 +323,13 @@ func onRenameIndex(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error)
return ver, errors.Trace(dbterror.ErrOptOnCacheTable.GenWithStackByArgs("Rename Index"))
}

idx := tblInfo.FindIndexByName(from.L)
idx.Name = to
if job.MultiSchemaInfo != nil && job.MultiSchemaInfo.Revertible {
job.MarkNonRevertible()
// Store the mark and enter the next DDL handling loop.
return updateVersionAndTableInfoWithCheck(d, t, job, tblInfo, false)
}

renameIndexes(tblInfo, from, to)
if ver, err = updateVersionAndTableInfo(d, t, job, tblInfo, true); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
Expand Down Expand Up @@ -1596,3 +1601,14 @@ func findIdxCol(idxInfo *model.IndexInfo, colName model.CIStr) int {
}
return -1
}

func renameIndexes(tblInfo *model.TableInfo, from, to model.CIStr) {
for _, idx := range tblInfo.Indices {
if idx.Name.L == from.L {
idx.Name = to
} else if isTempIdxInfo(idx, tblInfo) && getChangingIndexOriginName(idx) == from.O {
idx.Name.L = strings.Replace(idx.Name.L, from.L, to.L, 1)
idx.Name.O = strings.Replace(idx.Name.O, from.O, to.O, 1)
}
}
}
11 changes: 11 additions & 0 deletions ddl/multi_schema_change.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@ import (
)

func (d *ddl) MultiSchemaChange(ctx sessionctx.Context, ti ast.Ident) error {
if len(ctx.GetSessionVars().StmtCtx.MultiSchemaInfo.SubJobs) == 0 {
return nil
}
schema, t, err := d.getSchemaAndTableByIdent(ctx, ti)
if err != nil {
return errors.Trace(err)
Expand Down Expand Up @@ -223,6 +226,11 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error
}
}
}
case model.ActionRenameIndex:
from := job.Args[0].(model.CIStr)
to := job.Args[1].(model.CIStr)
info.AddIndexes = append(info.AddIndexes, to)
info.DropIndexes = append(info.DropIndexes, from)
case model.ActionModifyColumn:
newCol := *job.Args[0].(**model.ColumnInfo)
oldColName := job.Args[1].(model.CIStr)
Expand All @@ -236,6 +244,9 @@ func fillMultiSchemaInfo(info *model.MultiSchemaInfo, job *model.Job) (err error
if pos != nil && pos.Tp == ast.ColumnPositionAfter {
info.PositionColumns = append(info.PositionColumns, pos.RelativeColumn.Name)
}
case model.ActionSetDefaultValue:
col := job.Args[0].(*table.Column)
info.ModifyColumns = append(info.ModifyColumns, col.Name)
default:
return dbterror.ErrRunMultiSchemaChanges
}
Expand Down
259 changes: 259 additions & 0 deletions ddl/multi_schema_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,200 @@ func TestMultiSchemaChangeAddDropColumns(t *testing.T) {
tk.MustGetErrCode("alter table t add column c int default 3 after a, add column d int default 4 first, drop column a, drop column b;", errno.ErrUnsupportedDDLOperation)
}

func TestMultiSchemaChangeRenameColumns(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomain(t)
originHook := dom.DDL().GetHook()
defer clean()

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

// unsupported ddl operations
{
// Test add and rename to same column name
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t rename column b to c, add column c int", errno.ErrUnsupportedDDLOperation)

// Test add column related with rename column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t rename column b to c, add column e int after b", errno.ErrUnsupportedDDLOperation)

// Test drop and rename with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t drop column b, rename column b to c", errno.ErrUnsupportedDDLOperation)

// Test add index and rename with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2, index t(a, b));")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t rename column b to c, add index t1(a, b)", errno.ErrUnsupportedDDLOperation)
}

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2, index t(a, b));")
tk.MustExec("insert into t values ();")
tk.MustExec("alter table t rename column b to c, add column e int default 3")
tk.MustQuery("select c from t").Check(testkit.Rows("2"))
tk.MustQuery("select * from t").Check(testkit.Rows("1 2 3"))

// Test cancel job with rename columns
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2)")
tk.MustExec("insert into t values ()")
hook := newCancelJobHook(store, dom, func(job *model.Job) bool {
// Cancel job when the column 'c' is in write-reorg.
return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization
})
dom.DDL().SetHook(hook)
tk.MustGetErrCode("alter table t add column c int default 3, rename column b to d;", errno.ErrCancelledDDLJob)
dom.DDL().SetHook(originHook)
tk.MustQuery("select b from t").Check(testkit.Rows("2"))
tk.MustGetErrCode("select d from t", errno.ErrBadField)

// Test dml stmts when do rename
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2)")
tk.MustExec("insert into t values ()")
hook1 := &ddl.TestDDLCallback{Do: dom}
hook1.OnJobRunBeforeExported = func(job *model.Job) {
assert.Equal(t, model.ActionMultiSchemaChange, job.Type)
if job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization {
rs, _ := tk.Exec("select b from t")
assert.Equal(t, tk.ResultSetToResult(rs, "").Rows()[0][0], "2")
}
}
dom.DDL().SetHook(hook1)
tk.MustExec("alter table t add column c int default 3, rename column b to d;")
dom.DDL().SetHook(originHook)
tk.MustQuery("select d from t").Check(testkit.Rows("2"))
tk.MustGetErrCode("select b from t", errno.ErrBadField)
}

func TestMultiSchemaChangeAlterColumns(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomain(t)
originHook := dom.DDL().GetHook()
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

// unsupported ddl operations
{
// Test alter and drop with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t alter column b set default 3, drop column b", errno.ErrUnsupportedDDLOperation)

// Test alter and rename with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t alter column b set default 3, rename column b to c", errno.ErrUnsupportedDDLOperation)

// Test alter and drop modify same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t alter column b set default 3, modify column b double", errno.ErrUnsupportedDDLOperation)
}

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2, index t(a, b));")
tk.MustExec("insert into t values ();")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2"))
tk.MustExec("alter table t rename column a to c, alter column b set default 3;")
tk.MustExec("truncate table t;")
tk.MustExec("insert into t values ();")
tk.MustQuery("select * from t").Check(testkit.Rows("1 3"))

// Test cancel job with alter columns
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2)")
hook := newCancelJobHook(store, dom, func(job *model.Job) bool {
// Cancel job when the column 'a' is in write-reorg.
return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization
})
dom.DDL().SetHook(hook)
tk.MustGetErrCode("alter table t add column c int default 3, alter column b set default 3;", errno.ErrCancelledDDLJob)
dom.DDL().SetHook(originHook)
tk.MustExec("insert into t values ()")
tk.MustQuery("select * from t").Check(testkit.Rows("1 2"))

// Test dml stmts when do alter
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2)")
hook1 := &ddl.TestDDLCallback{Do: dom}
hook1.OnJobRunBeforeExported = func(job *model.Job) {
assert.Equal(t, model.ActionMultiSchemaChange, job.Type)
if job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteOnly {
tk.Exec("insert into t values ()")
}
}
dom.DDL().SetHook(hook1)
tk.MustExec("alter table t add column c int default 3, alter column b set default 3;")
dom.DDL().SetHook(originHook)
tk.MustQuery("select * from t").Check(testkit.Rows("1 2 3"))
}

func TestMultiSchemaChangeChangeColumns(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomain(t)
originHook := dom.DDL().GetHook()
defer clean()

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")

// unsupported ddl operations
{
// Test change and drop with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t change column b c double, drop column b", errno.ErrUnsupportedDDLOperation)

// Test change and add with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2);")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t change column b c double, add column c int", errno.ErrUnsupportedDDLOperation)

// Test add index and rename with same column
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2, index t(a, b));")
tk.MustExec("insert into t values ();")
tk.MustGetErrCode("alter table t change column b c double, add index t1(a, b)", errno.ErrUnsupportedDDLOperation)
}

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int default 1, b int default 2, index t(a, b));")
tk.MustExec("insert into t values ();")
tk.MustExec("alter table t rename column b to c, change column a e bigint default 3;")
tk.MustQuery("select e,c from t").Check(testkit.Rows("1 2"))
tk.MustExec("truncate table t;")
tk.MustExec("insert into t values ();")
tk.MustQuery("select e,c from t").Check(testkit.Rows("3 2"))

// Test cancel job with change columns
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2)")
tk.MustExec("insert into t values ()")
hook := newCancelJobHook(store, dom, func(job *model.Job) bool {
// Cancel job when the column 'c' is in write-reorg.
return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization
})
dom.DDL().SetHook(hook)
tk.MustGetErrCode("alter table t add column c int default 3, change column b d bigint default 4;", errno.ErrCancelledDDLJob)
dom.DDL().SetHook(originHook)
tk.MustQuery("select b from t").Check(testkit.Rows("2"))
tk.MustGetErrCode("select d from t", errno.ErrBadField)
}

func TestMultiSchemaChangeAddIndexes(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down Expand Up @@ -505,6 +699,55 @@ func TestMultiSchemaChangeAddDropIndexes(t *testing.T) {
tk.MustExec("admin check table t;")
}

func TestMultiSchemaChangeRenameIndexes(t *testing.T) {
store, dom, clean := testkit.CreateMockStoreAndDomain(t)
defer clean()
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
originHook := dom.DDL().GetHook()

// Test rename index.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int, c int, index t(a), index t1(b))")
tk.MustExec("alter table t rename index t to x, rename index t1 to x1")
tk.MustExec("select * from t use index (x);")
tk.MustExec("select * from t use index (x1);")
tk.MustGetErrCode("select * from t use index (t);", errno.ErrKeyDoesNotExist)
tk.MustGetErrCode("select * from t use index (t1);", errno.ErrKeyDoesNotExist)

// Test drop and rename same index.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int, c int, index t(a))")
tk.MustGetErrCode("alter table t drop index t, rename index t to t1", errno.ErrUnsupportedDDLOperation)

// Test add and rename to same index name.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int, b int, c int, index t(a))")
tk.MustGetErrCode("alter table t add index t1(b), rename index t to t1", errno.ErrUnsupportedDDLOperation)

// Test drop column with rename index.
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2, c int default 3, index t(a))")
tk.MustExec("insert into t values ();")
tk.MustExec("alter table t drop column a, rename index t to x")
tk.MustGetErrCode("select * from t use index (x);", errno.ErrKeyDoesNotExist)
tk.MustQuery("select * from t;").Check(testkit.Rows("2 3"))

// Test cancel job with renameIndex
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (a int default 1, b int default 2, index t(a))")
tk.MustExec("insert into t values ()")
hook := newCancelJobHook(store, dom, func(job *model.Job) bool {
// Cancel job when the column 'c' is in write-reorg.
return job.MultiSchemaInfo.SubJobs[0].SchemaState == model.StateWriteReorganization
})
dom.DDL().SetHook(hook)
tk.MustGetErrCode("alter table t add column c int default 3, rename index t to t1;", errno.ErrCancelledDDLJob)
dom.DDL().SetHook(originHook)
tk.MustQuery("select * from t use index (t);").Check(testkit.Rows("1 2"))
tk.MustGetErrCode("select * from t use index (t1);", errno.ErrKeyDoesNotExist)
}

func TestMultiSchemaChangeModifyColumns(t *testing.T) {
store, clean := testkit.CreateMockStore(t)
defer clean()
Expand Down Expand Up @@ -591,6 +834,22 @@ func TestMultiSchemaChangeModifyColumns(t *testing.T) {
tk.MustQuery("select * from t use index(i1, i2);").Check(testkit.Rows("1 3 2", "11 33 22"))
tk.MustExec("admin check table t;")

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t(a bigint null default '1761233443433596323', index t(a));")
tk.MustExec("insert into t set a = '-7184819032643664798';")
tk.MustGetErrCode("alter table t change column a b datetime null default '8972-12-24 10:56:03', rename index t to t1;", errno.ErrTruncatedWrongValue)

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int, b double, index i(a, b));")
tk.MustExec("alter table t rename index i to i1, change column b c int;")
tk.MustQuery("select count(*) from information_schema.TIDB_INDEXES where TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='i1';").Check(testkit.Rows("1"))

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int, b double, index i(a, b), index _Idx$_i(a, b));")
tk.MustExec("alter table t rename index i to i1, change column b c int;")
tk.MustQuery("select count(*) from information_schema.TIDB_INDEXES where TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='i1';").Check(testkit.Rows("1"))
tk.MustQuery("select count(*) from information_schema.TIDB_INDEXES where TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='_Idx$_i';").Check(testkit.Rows("1"))

tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int, _Col$_a double, index _Idx$_i(a, _Col$_a), index i(a, _Col$_a));")
tk.MustExec("alter table t modify column a tinyint;")
Expand Down

0 comments on commit 12c1721

Please sign in to comment.