Skip to content

Commit

Permalink
ddl, parser: support 'ALTER TABLE RENAME KEY TO' syntax (pingcap#6475)
Browse files Browse the repository at this point in the history
  • Loading branch information
spongedu authored and XuHuaiyu committed Jun 7, 2018
1 parent 66c4112 commit f56e130
Show file tree
Hide file tree
Showing 10 changed files with 142 additions and 7 deletions.
3 changes: 3 additions & 0 deletions ast/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -717,6 +717,7 @@ const (
AlterTableAlterColumn
AlterTableLock
AlterTableAlgorithm
AlterTableRenameIndex
AlterTableForce

// TODO: Add more actions
Expand Down Expand Up @@ -748,6 +749,8 @@ type AlterTableSpec struct {
Position *ColumnPosition
LockType LockType
Comment string
FromKey model.CIStr
ToKey model.CIStr
}

// Accept implements Node Accept interface.
Expand Down
37 changes: 37 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -999,6 +999,8 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A
err = d.RenameTable(ctx, ident, newIdent)
case ast.AlterTableDropPrimaryKey:
err = ErrUnsupportedModifyPrimaryKey.GenByArgs("drop")
case ast.AlterTableRenameIndex:
err = d.RenameIndex(ctx, ident, spec)
case ast.AlterTableOption:
for _, opt := range spec.Options {
switch opt.Tp {
Expand Down Expand Up @@ -1574,6 +1576,41 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a
return errors.Trace(err)
}

// RenameIndex renames an index.
// In TiDB, indexes are case-insensitive (so index 'a' and 'A" are considered the same index),
// but index names are case-sensitive (we can rename index 'a' to 'A')
func (d *ddl) RenameIndex(ctx sessionctx.Context, ident ast.Ident, spec *ast.AlterTableSpec) error {
is := d.infoHandle.Get()
schema, ok := is.SchemaByName(ident.Schema)
if !ok {
return infoschema.ErrDatabaseNotExists.GenByArgs(ident.Schema)
}

tb, err := is.TableByName(ident.Schema, ident.Name)
if err != nil {
return errors.Trace(infoschema.ErrTableNotExists.GenByArgs(ident.Schema, ident.Name))
}
duplicate, err := validateRenameIndex(spec.FromKey, spec.ToKey, tb.Meta())
if duplicate {
return nil
}
if err != nil {
return errors.Trace(err)
}

job := &model.Job{
SchemaID: schema.ID,
TableID: tb.Meta().ID,
Type: model.ActionRenameIndex,
BinlogInfo: &model.HistoryInfo{},
Args: []interface{}{spec.FromKey, spec.ToKey},
}

err = d.doDDLJob(ctx, job)
err = d.callHookOnChanged(err)
return errors.Trace(err)
}

// DropTable will proceed even if some table in the list does not exists.
func (d *ddl) DropTable(ctx sessionctx.Context, ti ast.Ident) (err error) {
is := d.GetInformationSchema()
Expand Down
23 changes: 23 additions & 0 deletions ddl/ddl_db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -268,6 +268,29 @@ func (s *testDBSuite) TestAddIndexWithPK(c *C) {
s.tk.MustQuery("select * from test_add_index_with_pk2").Check(testkit.Rows("1 1 1 1", "2 2 2 2"))
}

func (s *testDBSuite) TestRenameIndex(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use " + s.schemaName)
s.tk.MustExec("create table t (pk int primary key, c int default 1, c1 int default 1, unique key k1(c), key k2(c1))")

// Test rename success
s.tk.MustExec("alter table t rename index k1 to k3")
s.tk.MustExec("admin check index t k3")

// Test rename to the same name
s.tk.MustExec("alter table t rename index k3 to k3")
s.tk.MustExec("admin check index t k3")

// Test rename on non-exists keys
s.testErrorCode(c, "alter table t rename index x to x", mysql.ErrKeyDoesNotExist)

// Test rename on already-exists keys
s.testErrorCode(c, "alter table t rename index k3 to k2", mysql.ErrDupKeyName)

s.tk.MustExec("alter table t rename index k2 to K2")
s.testErrorCode(c, "alter table t rename key k3 to K2", mysql.ErrDupKeyName)
}

func (s *testDBSuite) testGetTable(c *C, name string) table.Table {
ctx := s.s.(sessionctx.Context)
dom := domain.GetDomain(ctx)
Expand Down
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -389,6 +389,8 @@ func (d *ddl) runDDLJob(t *meta.Meta, job *model.Job) (ver int64, err error) {
ver, err = d.onShardRowID(t, job)
case model.ActionModifyTableComment:
ver, err = d.onModifyTableComment(t, job)
case model.ActionRenameIndex:
ver, err = d.onRenameIndex(t, job)
default:
// Invalid job, cancel it.
job.State = model.JobStateCancelled
Expand Down
18 changes: 18 additions & 0 deletions ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -174,6 +175,23 @@ func dropIndexColumnFlag(tblInfo *model.TableInfo, indexInfo *model.IndexInfo) {
}
}

func validateRenameIndex(from, to model.CIStr, tbl *model.TableInfo) (ignore bool, err error) {
if fromIdx := findIndexByName(from.L, tbl.Indices); fromIdx == nil {
return false, errors.Trace(infoschema.ErrKeyNotExists.GenByArgs(from.O, tbl.Name))
}
// Take case-sensitivity into account, if `FromKey` and `ToKey` are the same, nothing need to be changed
if from.O == to.O {
return true, nil
}
// If spec.FromKey.L == spec.ToKey.L, we operate on the same index(case-insensitive) and change its name (case-sensitive)
// e.g: from `inDex` to `IndEX`. Otherwise, we try to rename an index to another different index which already exists,
// that's illegal by rule.
if toIdx := findIndexByName(to.L, tbl.Indices); toIdx != nil && from.L != to.L {
return false, errors.Trace(infoschema.ErrKeyNameDuplicate.GenByArgs(toIdx.Name.O))
}
return false, nil
}

func (d *ddl) onCreateIndex(t *meta.Meta, job *model.Job) (ver int64, err error) {
// Handle the rolling back job.
if job.IsRollingback() {
Expand Down
31 changes: 31 additions & 0 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -357,6 +357,37 @@ func (d *ddl) onModifyTableComment(t *meta.Meta, job *model.Job) (ver int64, _ e
return ver, nil
}

func (d *ddl) onRenameIndex(t *meta.Meta, job *model.Job) (ver int64, _ error) {
var from, to model.CIStr
if err := job.DecodeArgs(&from, &to); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
tblInfo, err := getTableInfo(t, job, job.SchemaID)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

// Double check. See function `RenameIndex` in ddl_api.go
duplicate, err := validateRenameIndex(from, to, tblInfo)
if duplicate {
return ver, nil
}
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
idx := findIndexByName(from.L, tblInfo.Indices)
idx.Name = to
if ver, err = updateVersionAndTableInfo(t, job, tblInfo, true); err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
job.FinishTableJob(model.JobStateDone, model.StatePublic, ver, tblInfo)
return ver, nil
}

func checkTableNotExists(t *meta.Meta, job *model.Job, schemaID int64, tableName string) error {
// Check this table's database.
tables, err := t.ListTables(schemaID)
Expand Down
22 changes: 15 additions & 7 deletions infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -50,6 +50,10 @@ var (
ErrColumnExists = terror.ClassSchema.New(codeColumnExists, "Duplicate column name '%s'")
// ErrIndexExists returns for index already exists.
ErrIndexExists = terror.ClassSchema.New(codeIndexExists, "Duplicate Index")
// ErrKeyNameDuplicate returns for index duplicate when rename index.
ErrKeyNameDuplicate = terror.ClassSchema.New(codeKeyNameDuplicate, "Duplicate key name '%s'")
// ErrKeyNotExists returns for index not exists.
ErrKeyNotExists = terror.ClassSchema.New(codeKeyNotExists, "Key '%s' doesn't exist in table '%s'")
// ErrMultiplePriKey returns for multiple primary keys.
ErrMultiplePriKey = terror.ClassSchema.New(codeMultiplePriKey, "Multiple primary key defined")
// ErrTooManyKeyParts returns for too many key parts.
Expand Down Expand Up @@ -279,13 +283,15 @@ const (
codeForeignKeyNotExists = 1091
codeWrongFkDef = 1239

codeDatabaseExists = 1007
codeTableExists = 1050
codeBadTable = 1051
codeColumnExists = 1060
codeIndexExists = 1831
codeMultiplePriKey = 1068
codeTooManyKeyParts = 1070
codeDatabaseExists = 1007
codeTableExists = 1050
codeBadTable = 1051
codeColumnExists = 1060
codeIndexExists = 1831
codeMultiplePriKey = 1068
codeTooManyKeyParts = 1070
codeKeyNameDuplicate = 1061
codeKeyNotExists = 1176
)

func init() {
Expand All @@ -304,6 +310,8 @@ func init() {
codeIndexExists: mysql.ErrDupIndex,
codeMultiplePriKey: mysql.ErrMultiplePriKey,
codeTooManyKeyParts: mysql.ErrTooManyKeyParts,
codeKeyNameDuplicate: mysql.ErrDupKeyName,
codeKeyNotExists: mysql.ErrKeyDoesNotExist,
}
terror.ErrClassToMySQLCodes[terror.ClassSchema] = schemaMySQLErrCodes
initInfoSchemaDB()
Expand Down
2 changes: 2 additions & 0 deletions model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,6 +47,7 @@ const (
ActionSetDefaultValue ActionType = 15
ActionShardRowID ActionType = 16
ActionModifyTableComment ActionType = 17
ActionRenameIndex ActionType = 18
)

var actionMap = map[ActionType]string{
Expand All @@ -67,6 +68,7 @@ var actionMap = map[ActionType]string{
ActionSetDefaultValue: "set default value",
ActionShardRowID: "shard row ID",
ActionModifyTableComment: "modify table comment",
ActionRenameIndex: "rename index",
}

// String return current ddl action in string
Expand Down
8 changes: 8 additions & 0 deletions parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -1040,6 +1040,14 @@ AlterTableSpec:
NewTable: $3.(*ast.TableName),
}
}
| "RENAME" KeyOrIndex Identifier "TO" Identifier
{
$$ = &ast.AlterTableSpec{
Tp: ast.AlterTableRenameIndex,
FromKey: model.NewCIStr($3),
ToKey: model.NewCIStr($5),
}
}
| LockClause
{
$$ = &ast.AlterTableSpec{
Expand Down
3 changes: 3 additions & 0 deletions parser/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1632,6 +1632,9 @@ func (s *testParserSuite) TestDDL(c *C) {
{"ALTER TABLE t CONVERT TO CHARSET utf8 COLLATE utf8_bin;", true},
{"ALTER TABLE t FORCE", true},
{"ALTER TABLE t DROP INDEX;", false},
// For #6405
{"ALTER TABLE t RENAME KEY a TO b;", true},
{"ALTER TABLE t RENAME INDEX a TO b;", true},

// For create index statement
{"CREATE INDEX idx ON t (a)", true},
Expand Down

0 comments on commit f56e130

Please sign in to comment.