From e882210856594edb3a8d096c2bc4c28357ce6b3e Mon Sep 17 00:00:00 2001 From: winkyao Date: Wed, 24 Oct 2018 17:12:13 +0800 Subject: [PATCH 01/13] ddl: support altering the other charset to utf8 or utf8mb4 --- ast/ddl.go | 2 + ddl/column_test.go | 2 +- ddl/db_change_test.go | 2 +- ddl/db_integration_test.go | 50 ++++++++++++++ ddl/db_test.go | 93 ++++++++++++++++++++++++++ ddl/ddl.go | 10 +-- ddl/ddl_api.go | 131 ++++++++++++++++++++++++++++++++++--- ddl/ddl_worker.go | 2 + ddl/table.go | 22 +++++++ model/ddl.go | 43 ++++++------ 10 files changed, 322 insertions(+), 35 deletions(-) diff --git a/ast/ddl.go b/ast/ddl.go index 340027e881c6f..4bc8a333419f2 100644 --- a/ast/ddl.go +++ b/ast/ddl.go @@ -682,6 +682,8 @@ type TableOption struct { Tp TableOptionType StrValue string UintValue uint64 + // Skipped indicate the TableOption can be skipped. + Skipped bool } // ColumnPositionType is the type for ColumnPosition. diff --git a/ddl/column_test.go b/ddl/column_test.go index 4eea2c9b862d0..c023a9142c969 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -915,7 +915,7 @@ func (s *testColumnSuite) TestModifyColumn(c *C) { {"int", "int unsigned", errUnsupportedModifyColumn.GenWithStackByArgs("length 10 is less than origin 11")}, {"varchar(10)", "text", nil}, {"varbinary(10)", "blob", nil}, - {"text", "blob", errUnsupportedModifyColumn.GenWithStackByArgs("charset binary not match origin utf8")}, + {"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8 to binary")}, {"varchar(10)", "varchar(8)", errUnsupportedModifyColumn.GenWithStackByArgs("length 8 is less than origin 10")}, {"varchar(10)", "varchar(11)", nil}, {"varchar(10) character set utf8 collate utf8_bin", "varchar(10) character set utf8", nil}, diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index bac31f693cd78..8b98d370cd748 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -376,7 +376,7 @@ func (s *testStateChangeSuite) TestAppendEnum(c *C) { c.Assert(err.Error(), Equals, "[ddl:203]unsupported modify column the number of enum column's elements is less than the original: 2") failAlterTableSQL2 := "alter table t change c2 c2 int default 0" _, err = s.se.Execute(context.Background(), failAlterTableSQL2) - c.Assert(err.Error(), Equals, "[ddl:203]unsupported modify column charset binary not match origin utf8") + c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8 to binary") alterTableSQL := "alter table t change c2 c2 enum('N','Y','A') DEFAULT 'A'" _, err = s.se.Execute(context.Background(), alterTableSQL) c.Assert(err, IsNil) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 77f03ba8ad119..f46fec5f7ffd7 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -152,6 +152,56 @@ func (s *testIntegrationSuite) TestEndIncluded(c *C) { tk.MustExec("admin check table t") } +func (s *testIntegrationSuite) TestChangingCharsetToUtf8(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("USE test") + tk.MustExec("create table t(a char(10) charset latin1)") + tk.MustExec("alter table t modify column a char(10) charset latin1") + tk.MustExec("alter table t modify column a char(10) charset utf8") + tk.MustExec("alter table t modify column a char(10) charset utf8mb4") + rs, err := tk.Exec("alter table t modify column a char(10) charset utf8mb4 collate utf8bin") + if rs != nil { + rs.Close() + } + c.Assert(err, NotNil) + tk.MustExec("alter table t modify column a char(10) charset utf8mb4 collate utf8mb4_bin") + rs, err = tk.Exec("alter table t modify column a char(10) charset utf8 collate utf8_bin") + if rs != nil { + rs.Close() + } + + c.Assert(err, NotNil) + tk.MustExec("alter table t modify column a char(10) charset utf8mb4 collate utf8mb4_general_ci") +} + +func (s *testIntegrationSuite) TestChangingTableCharset(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("USE test") + tk.MustExec("create table t(a char(10)) charset latin1 collate latin1_bin") + rs, err := tk.Exec("alter table t charset gbk") + if rs != nil { + rs.Close() + } + c.Assert(err.Error(), Equals, "Unknown charset gbk") + tk.MustExec("alter table t charset utf8") + tk.MustExec("alter table t charset utf8 collate utf8_bin") + rs, err = tk.Exec("alter table t charset utf8 collate latin1_bin") + if rs != nil { + rs.Close() + } + c.Assert(err, NotNil) + tk.MustExec("alter table t charset utf8mb4") + tk.MustExec("alter table t charset utf8mb4 collate utf8mb4_bin") + + rs, err = tk.Exec("alter table t charset utf8 collate utf8_bin") + if rs != nil { + rs.Close() + } + c.Assert(err, NotNil) +} + func newStoreWithBootstrap() (kv.Storage, *domain.Domain, error) { store, err := mockstore.NewMockTikvStore() if err != nil { diff --git a/ddl/db_test.go b/ddl/db_test.go index e87024fdf63b2..9f3edf4c27b10 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -44,6 +44,7 @@ import ( "github.com/pingcap/tidb/terror" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" + "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" @@ -3576,3 +3577,95 @@ func (s *testDBSuite) TestPartitionAddIndex(c *C) { tk.MustExec("admin check table partition_add_idx") tk.MustExec("drop table partition_add_idx") } + +func (s *testDBSuite) TestAlterTableCharset(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database test_charset") + defer tk.MustExec("drop database test_charset") + tk.MustExec("use test_charset") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a int) charset latin1") + ctx := tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + t, err := is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(err, IsNil) + c.Assert(t.Meta().Charset, Equals, "latin1") + defCollate, err := charset.GetDefaultCollation("latin1") + c.Assert(err, IsNil) + c.Assert(t.Meta().Collate, Equals, defCollate) + + tk.MustExec("alter table t charset utf8") + is = domain.GetDomain(ctx).InfoSchema() + t, err = is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(t.Meta().Charset, Equals, "utf8") + defCollate, err = charset.GetDefaultCollation("utf8") + c.Assert(err, IsNil) + c.Assert(t.Meta().Collate, Equals, defCollate) + + tk.MustExec("alter table t charset utf8mb4 collate utf8mb4_general_ci") + is = domain.GetDomain(ctx).InfoSchema() + t, err = is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(t.Meta().Charset, Equals, "utf8mb4") + c.Assert(t.Meta().Collate, Equals, "utf8mb4_general_ci") + + rs, err := tk.Exec("alter table t charset utf8") + if rs != nil { + rs.Close() + } + + c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8mb4 to utf8") +} + +func (s *testDBSuite) TestAlterColumnCharset(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("create database test_charset") + defer tk.MustExec("drop database test_charset") + tk.MustExec("use test_charset") + tk.MustExec("drop table if exists t") + tk.MustExec("create table t(a char(10) charset latin1)") + ctx := tk.Se.(sessionctx.Context) + is := domain.GetDomain(ctx).InfoSchema() + t, err := is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(err, IsNil) + col := model.FindColumnInfo(t.Meta().Columns, "a") + c.Assert(col, NotNil) + c.Assert(col.Charset, Equals, "latin1") + defCollate, err := charset.GetDefaultCollation("latin1") + c.Assert(err, IsNil) + c.Assert(col.Collate, Equals, defCollate) + + tk.MustExec("alter table t modify column a char(10) charset utf8") + is = domain.GetDomain(ctx).InfoSchema() + t, err = is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(err, IsNil) + col = model.FindColumnInfo(t.Meta().Columns, "a") + c.Assert(col, NotNil) + c.Assert(col.Charset, Equals, "utf8") + defCollate, err = charset.GetDefaultCollation("utf8") + c.Assert(err, IsNil) + c.Assert(col.Collate, Equals, defCollate) + + tk.MustExec("alter table t modify column a char(10) charset utf8 collate utf8_general_ci") + is = domain.GetDomain(ctx).InfoSchema() + t, err = is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(err, IsNil) + col = model.FindColumnInfo(t.Meta().Columns, "a") + c.Assert(col, NotNil) + c.Assert(col.Charset, Equals, "utf8") + c.Assert(col.Collate, Equals, "utf8_general_ci") + + tk.MustExec("alter table t modify column a char(10) charset utf8mb4 collate utf8mb4_general_ci") + is = domain.GetDomain(ctx).InfoSchema() + t, err = is.TableByName(model.NewCIStr("test_charset"), model.NewCIStr("t")) + c.Assert(err, IsNil) + col = model.FindColumnInfo(t.Meta().Columns, "a") + c.Assert(col, NotNil) + c.Assert(col.Charset, Equals, "utf8mb4") + c.Assert(col.Collate, Equals, "utf8mb4_general_ci") + + rs, err := tk.Exec("alter table t modify column a char(10) charset utf8") + if rs != nil { + rs.Close() + } + c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8mb4 to utf8") +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 6114e51950142..7d06f6a219684 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -83,10 +83,11 @@ var ( errInvalidStoreVer = terror.ClassDDL.New(codeInvalidStoreVer, "invalid storage current version") // We don't support dropping column with index covered now. - errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index") - errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column") - errUnsupportedModifyColumn = terror.ClassDDL.New(codeUnsupportedModifyColumn, "unsupported modify column %s") - errUnsupportedPKHandle = terror.ClassDDL.New(codeUnsupportedDropPKHandle, + errCantDropColWithIndex = terror.ClassDDL.New(codeCantDropColWithIndex, "can't drop column with index") + errUnsupportedAddColumn = terror.ClassDDL.New(codeUnsupportedAddColumn, "unsupported add column") + errUnsupportedModifyColumn = terror.ClassDDL.New(codeUnsupportedModifyColumn, "unsupported modify column %s") + errUnsupportedModifyCharset = terror.ClassDDL.New(codeUnsupportedModifyCharset, "unsupported modify %s") + errUnsupportedPKHandle = terror.ClassDDL.New(codeUnsupportedDropPKHandle, "unsupported drop integer primary key") errUnsupportedCharset = terror.ClassDDL.New(codeUnsupportedCharset, "unsupported charset %s collate %s") @@ -577,6 +578,7 @@ const ( codeUnsupportedCharset = 205 codeUnsupportedModifyPrimaryKey = 206 codeUnsupportedShardRowIDBits = 207 + codeUnsupportedModifyCharset = 208 codeFileNotFound = 1017 codeErrorOnRename = 1025 diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 3c6aa44f9c60e..3f2e22aff2514 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -999,6 +999,17 @@ func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64) error { return nil } +func setDefaultTableCharsetAndCollation(tbInfo *model.TableInfo) (err error) { + if len(tbInfo.Charset) == 0 { + tbInfo.Charset = mysql.DefaultCharset + } + + if len(tbInfo.Collate) == 0 { + tbInfo.Collate, err = charset.GetDefaultCollation(tbInfo.Charset) + } + return +} + // handleTableOptions updates tableInfo according to table options. func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) error { for _, op := range options { @@ -1023,6 +1034,8 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err } } } + + setDefaultTableCharsetAndCollation(tbInfo) return nil } @@ -1042,6 +1055,28 @@ func isIgnorableSpec(tp ast.AlterTableType) bool { return tp == ast.AlterTableLock || tp == ast.AlterTableAlgorithm } +// getCharsetAndCollateInTableOption will iterate the charset and collate in the options, +// and returns the last charset and collate in options. +func getCharsetAndCollateInTableOption(startIdx int, options []*ast.TableOption) (charset, collate string) { + for i := startIdx; i < len(options); i++ { + opt := options[i] + // we set the charset to the last option. example: alter table t charset latin1 charset utf8 collate utf8_bin; + // the charset will be utf8, collate will be utf8_bin + switch opt.Tp { + case ast.TableOptionCharset: + charset = opt.StrValue + // this opt is handled, so we can skipped in the next iteration. + options[i].Skipped = true + case ast.TableOptionCollate: + collate = opt.StrValue + // this opt is handled, so we can skipped in the next iteration. + options[i].Skipped = true + } + } + + return charset, collate +} + func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) { // Only handle valid specs. validSpecs := make([]*ast.AlterTableSpec, 0, len(specs)) @@ -1103,7 +1138,11 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A case ast.AlterTableRenameIndex: err = d.RenameIndex(ctx, ident, spec) case ast.AlterTableOption: - for _, opt := range spec.Options { + for i, opt := range spec.Options { + if opt.Skipped { + continue + } + switch opt.Tp { case ast.TableOptionShardRowID: if opt.UintValue > shardRowIDBitsMax { @@ -1115,7 +1154,11 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A case ast.TableOptionComment: spec.Comment = opt.StrValue err = d.AlterTableComment(ctx, ident, spec) + case ast.TableOptionCharset, ast.TableOptionCollate: + toCharset, toCollate := getCharsetAndCollateInTableOption(i, spec.Options) + err = d.AlterTableCharsetAndCollate(ctx, ident, toCharset, toCollate) } + if err != nil { return errors.Trace(err) } @@ -1419,6 +1462,29 @@ func (d *ddl) DropColumn(ctx sessionctx.Context, ti ast.Ident, colName model.CIS return errors.Trace(err) } +// modifiableCharsetAndCollation returns error when the charset or collation is not modifiable. +func modifiableCharsetAndCollation(toCharset, toCollate, origCharset, origCollate string) error { + if !charset.ValidCharsetAndCollation(toCharset, toCollate) { + return ErrUnknownCharacterSet.GenWithStackByArgs(toCharset, toCollate) + } + + if toCharset == charset.CharsetUTF8MB4 || (toCharset == charset.CharsetUTF8 && origCharset != charset.CharsetUTF8MB4) { + // TiDB treats all the data as utf8mb4, so we support changing the charset to utf8mb4. + // And not allow to change utf8mb4 to utf8. + return nil + } + + if toCharset != origCharset { + msg := fmt.Sprintf("charset from %s to %s", origCharset, toCharset) + return errUnsupportedModifyCharset.GenWithStackByArgs(msg) + } + if toCollate != origCollate { + msg := fmt.Sprintf("collate from %s to %s", origCollate, toCollate) + return errUnsupportedModifyCharset.GenWithStackByArgs(msg) + } + return nil +} + // modifiable checks if the 'origin' type can be modified to 'to' type with out the need to // change or check existing data in the table. // It returns true if the two types has the same Charset and Collation, the same sign, both are @@ -1432,14 +1498,10 @@ func modifiable(origin *types.FieldType, to *types.FieldType) error { msg := fmt.Sprintf("decimal %d is less than origin %d", to.Decimal, origin.Decimal) return errUnsupportedModifyColumn.GenWithStackByArgs(msg) } - if to.Charset != origin.Charset { - msg := fmt.Sprintf("charset %s not match origin %s", to.Charset, origin.Charset) - return errUnsupportedModifyColumn.GenWithStackByArgs(msg) - } - if to.Collate != origin.Collate { - msg := fmt.Sprintf("collate %s not match origin %s", to.Collate, origin.Collate) - return errUnsupportedModifyColumn.GenWithStackByArgs(msg) + if err := modifiableCharsetAndCollation(to.Charset, to.Collate, origin.Charset, origin.Collate); err != nil { + return errors.Trace(err) } + toUnsigned := mysql.HasUnsignedFlag(to.Flag) originUnsigned := mysql.HasUnsignedFlag(origin.Flag) if originUnsigned != toUnsigned { @@ -1802,6 +1864,59 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a return errors.Trace(err) } +// AlterTableCharset changes the table charset and collate. +func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset string, toCollate string) error { + if toCharset == "" && toCollate == "" { + return errors.Errorf("toCharset and toCollate can't be empty") + } + + is := d.infoHandle.Get() + schema, ok := is.SchemaByName(ident.Schema) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(ident.Schema) + } + + tb, err := is.TableByName(ident.Schema, ident.Name) + if err != nil { + return errors.Trace(infoschema.ErrTableNotExists.GenWithStackByArgs(ident.Schema, ident.Name)) + } + + origCharset := tb.Meta().Charset + origCollate := tb.Meta().Collate + if toCharset == "" { + // charset does not change. + toCharset = origCharset + } + + if toCollate == "" { + // get the default collation of the charset. + toCollate, err = charset.GetDefaultCollation(toCharset) + if err != nil { + return errors.Trace(err) + } + } + + if origCharset == toCharset && origCollate == toCollate { + // nothing to do. + return nil + } + + if err := modifiableCharsetAndCollation(toCharset, toCollate, origCharset, origCollate); err != nil { + return errors.Trace(err) + } + + job := &model.Job{ + SchemaID: schema.ID, + TableID: tb.Meta().ID, + Type: model.ActionModifyTableCharsetAndCollate, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{toCharset, toCollate}, + } + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + 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') diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index d015911a3c80b..dc26096041fbf 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -507,6 +507,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, ver, err = onModifyTableComment(t, job) case model.ActionAddTablePartition: ver, err = onAddTablePartition(t, job) + case model.ActionModifyTableCharsetAndCollate: + ver, err = onModifyTableCharsetAndCollate(t, job) default: // Invalid job, cancel it. job.State = model.JobStateCancelled diff --git a/ddl/table.go b/ddl/table.go index 930f871ff43d2..5ac919328b8f5 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -382,6 +382,28 @@ func onModifyTableComment(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, nil } +func onModifyTableCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _ error) { + var toCharset, toCollate string + if err := job.DecodeArgs(&toCharset, &toCollate); err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + tblInfo, err := getTableInfo(t, job, job.SchemaID) + if err != nil { + return ver, errors.Trace(err) + } + + tblInfo.Charset = toCharset + tblInfo.Collate = toCollate + ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) + if err != nil { + 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) diff --git a/model/ddl.go b/model/ddl.go index 23db9df3e2d90..93d1fa8e69602 100644 --- a/model/ddl.go +++ b/model/ddl.go @@ -29,27 +29,28 @@ type ActionType byte // List DDL actions. const ( - ActionNone ActionType = 0 - ActionCreateSchema ActionType = 1 - ActionDropSchema ActionType = 2 - ActionCreateTable ActionType = 3 - ActionDropTable ActionType = 4 - ActionAddColumn ActionType = 5 - ActionDropColumn ActionType = 6 - ActionAddIndex ActionType = 7 - ActionDropIndex ActionType = 8 - ActionAddForeignKey ActionType = 9 - ActionDropForeignKey ActionType = 10 - ActionTruncateTable ActionType = 11 - ActionModifyColumn ActionType = 12 - ActionRebaseAutoID ActionType = 13 - ActionRenameTable ActionType = 14 - ActionSetDefaultValue ActionType = 15 - ActionShardRowID ActionType = 16 - ActionModifyTableComment ActionType = 17 - ActionRenameIndex ActionType = 18 - ActionAddTablePartition ActionType = 19 - ActionDropTablePartition ActionType = 20 + ActionNone ActionType = 0 + ActionCreateSchema ActionType = 1 + ActionDropSchema ActionType = 2 + ActionCreateTable ActionType = 3 + ActionDropTable ActionType = 4 + ActionAddColumn ActionType = 5 + ActionDropColumn ActionType = 6 + ActionAddIndex ActionType = 7 + ActionDropIndex ActionType = 8 + ActionAddForeignKey ActionType = 9 + ActionDropForeignKey ActionType = 10 + ActionTruncateTable ActionType = 11 + ActionModifyColumn ActionType = 12 + ActionRebaseAutoID ActionType = 13 + ActionRenameTable ActionType = 14 + ActionSetDefaultValue ActionType = 15 + ActionShardRowID ActionType = 16 + ActionModifyTableComment ActionType = 17 + ActionRenameIndex ActionType = 18 + ActionAddTablePartition ActionType = 19 + ActionDropTablePartition ActionType = 20 + ActionModifyTableCharsetAndCollate ActionType = 21 ) // AddIndexStr is a string related to the operation of "add index". From a54cb1ebf505246a69320ff577ae83109e442560 Mon Sep 17 00:00:00 2001 From: winkyao Date: Wed, 24 Oct 2018 17:35:30 +0800 Subject: [PATCH 02/13] fix ci --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 3f2e22aff2514..beaaf6385692f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1901,7 +1901,7 @@ func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Iden return nil } - if err := modifiableCharsetAndCollation(toCharset, toCollate, origCharset, origCollate); err != nil { + if err = modifiableCharsetAndCollation(toCharset, toCollate, origCharset, origCollate); err != nil { return errors.Trace(err) } From 1069ffe76d87f28d3c2f9b38fcf72dee7788ac79 Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 5 Nov 2018 17:43:47 +0800 Subject: [PATCH 03/13] remove mistakenly added files --- ast/ddl.go | 895 --------------------------------------------------- model/ddl.go | 385 ---------------------- 2 files changed, 1280 deletions(-) delete mode 100644 ast/ddl.go delete mode 100644 model/ddl.go diff --git a/ast/ddl.go b/ast/ddl.go deleted file mode 100644 index a8e9043380b4d..0000000000000 --- a/ast/ddl.go +++ /dev/null @@ -1,895 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package ast - -import ( - "github.com/pingcap/tidb/model" - "github.com/pingcap/tidb/parser/types" -) - -var ( - _ DDLNode = &AlterTableStmt{} - _ DDLNode = &CreateDatabaseStmt{} - _ DDLNode = &CreateIndexStmt{} - _ DDLNode = &CreateTableStmt{} - _ DDLNode = &CreateViewStmt{} - _ DDLNode = &DropDatabaseStmt{} - _ DDLNode = &DropIndexStmt{} - _ DDLNode = &DropTableStmt{} - _ DDLNode = &RenameTableStmt{} - _ DDLNode = &TruncateTableStmt{} - - _ Node = &AlterTableSpec{} - _ Node = &ColumnDef{} - _ Node = &ColumnOption{} - _ Node = &ColumnPosition{} - _ Node = &Constraint{} - _ Node = &IndexColName{} - _ Node = &ReferenceDef{} -) - -// CharsetOpt is used for parsing charset option from SQL. -type CharsetOpt struct { - Chs string - Col string -} - -// DatabaseOptionType is the type for database options. -type DatabaseOptionType int - -// Database option types. -const ( - DatabaseOptionNone DatabaseOptionType = iota - DatabaseOptionCharset - DatabaseOptionCollate -) - -// DatabaseOption represents database option. -type DatabaseOption struct { - Tp DatabaseOptionType - Value string -} - -// CreateDatabaseStmt is a statement to create a database. -// See https://dev.mysql.com/doc/refman/5.7/en/create-database.html -type CreateDatabaseStmt struct { - ddlNode - - IfNotExists bool - Name string - Options []*DatabaseOption -} - -// Accept implements Node Accept interface. -func (n *CreateDatabaseStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*CreateDatabaseStmt) - return v.Leave(n) -} - -// DropDatabaseStmt is a statement to drop a database and all tables in the database. -// See https://dev.mysql.com/doc/refman/5.7/en/drop-database.html -type DropDatabaseStmt struct { - ddlNode - - IfExists bool - Name string -} - -// Accept implements Node Accept interface. -func (n *DropDatabaseStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*DropDatabaseStmt) - return v.Leave(n) -} - -// IndexColName is used for parsing index column name from SQL. -type IndexColName struct { - node - - Column *ColumnName - Length int -} - -// Accept implements Node Accept interface. -func (n *IndexColName) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*IndexColName) - node, ok := n.Column.Accept(v) - if !ok { - return n, false - } - n.Column = node.(*ColumnName) - return v.Leave(n) -} - -// ReferenceDef is used for parsing foreign key reference option from SQL. -// See http://dev.mysql.com/doc/refman/5.7/en/create-table-foreign-keys.html -type ReferenceDef struct { - node - - Table *TableName - IndexColNames []*IndexColName - OnDelete *OnDeleteOpt - OnUpdate *OnUpdateOpt -} - -// Accept implements Node Accept interface. -func (n *ReferenceDef) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*ReferenceDef) - node, ok := n.Table.Accept(v) - if !ok { - return n, false - } - n.Table = node.(*TableName) - for i, val := range n.IndexColNames { - node, ok = val.Accept(v) - if !ok { - return n, false - } - n.IndexColNames[i] = node.(*IndexColName) - } - onDelete, ok := n.OnDelete.Accept(v) - if !ok { - return n, false - } - n.OnDelete = onDelete.(*OnDeleteOpt) - onUpdate, ok := n.OnUpdate.Accept(v) - if !ok { - return n, false - } - n.OnUpdate = onUpdate.(*OnUpdateOpt) - return v.Leave(n) -} - -// ReferOptionType is the type for refer options. -type ReferOptionType int - -// Refer option types. -const ( - ReferOptionNoOption ReferOptionType = iota - ReferOptionRestrict - ReferOptionCascade - ReferOptionSetNull - ReferOptionNoAction -) - -// String implements fmt.Stringer interface. -func (r ReferOptionType) String() string { - switch r { - case ReferOptionRestrict: - return "RESTRICT" - case ReferOptionCascade: - return "CASCADE" - case ReferOptionSetNull: - return "SET NULL" - case ReferOptionNoAction: - return "NO ACTION" - } - return "" -} - -// OnDeleteOpt is used for optional on delete clause. -type OnDeleteOpt struct { - node - ReferOpt ReferOptionType -} - -// Accept implements Node Accept interface. -func (n *OnDeleteOpt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*OnDeleteOpt) - return v.Leave(n) -} - -// OnUpdateOpt is used for optional on update clause. -type OnUpdateOpt struct { - node - ReferOpt ReferOptionType -} - -// Accept implements Node Accept interface. -func (n *OnUpdateOpt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*OnUpdateOpt) - return v.Leave(n) -} - -// ColumnOptionType is the type for ColumnOption. -type ColumnOptionType int - -// ColumnOption types. -const ( - ColumnOptionNoOption ColumnOptionType = iota - ColumnOptionPrimaryKey - ColumnOptionNotNull - ColumnOptionAutoIncrement - ColumnOptionDefaultValue - ColumnOptionUniqKey - ColumnOptionNull - ColumnOptionOnUpdate // For Timestamp and Datetime only. - ColumnOptionFulltext - ColumnOptionComment - ColumnOptionGenerated - ColumnOptionReference -) - -// ColumnOption is used for parsing column constraint info from SQL. -type ColumnOption struct { - node - - Tp ColumnOptionType - // Expr is used for ColumnOptionDefaultValue/ColumnOptionOnUpdateColumnOptionGenerated. - // For ColumnOptionDefaultValue or ColumnOptionOnUpdate, it's the target value. - // For ColumnOptionGenerated, it's the target expression. - Expr ExprNode - // Stored is only for ColumnOptionGenerated, default is false. - Stored bool - // Refer is used for foreign key. - Refer *ReferenceDef -} - -// Accept implements Node Accept interface. -func (n *ColumnOption) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*ColumnOption) - if n.Expr != nil { - node, ok := n.Expr.Accept(v) - if !ok { - return n, false - } - n.Expr = node.(ExprNode) - } - return v.Leave(n) -} - -// IndexOption is the index options. -// KEY_BLOCK_SIZE [=] value -// | index_type -// | WITH PARSER parser_name -// | COMMENT 'string' -// See http://dev.mysql.com/doc/refman/5.7/en/create-table.html -type IndexOption struct { - node - - KeyBlockSize uint64 - Tp model.IndexType - Comment string -} - -// Accept implements Node Accept interface. -func (n *IndexOption) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*IndexOption) - return v.Leave(n) -} - -// ConstraintType is the type for Constraint. -type ConstraintType int - -// ConstraintTypes -const ( - ConstraintNoConstraint ConstraintType = iota - ConstraintPrimaryKey - ConstraintKey - ConstraintIndex - ConstraintUniq - ConstraintUniqKey - ConstraintUniqIndex - ConstraintForeignKey - ConstraintFulltext -) - -// Constraint is constraint for table definition. -type Constraint struct { - node - - Tp ConstraintType - Name string - - Keys []*IndexColName // Used for PRIMARY KEY, UNIQUE, ...... - - Refer *ReferenceDef // Used for foreign key. - - Option *IndexOption // Index Options -} - -// Accept implements Node Accept interface. -func (n *Constraint) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*Constraint) - for i, val := range n.Keys { - node, ok := val.Accept(v) - if !ok { - return n, false - } - n.Keys[i] = node.(*IndexColName) - } - if n.Refer != nil { - node, ok := n.Refer.Accept(v) - if !ok { - return n, false - } - n.Refer = node.(*ReferenceDef) - } - if n.Option != nil { - node, ok := n.Option.Accept(v) - if !ok { - return n, false - } - n.Option = node.(*IndexOption) - } - return v.Leave(n) -} - -// ColumnDef is used for parsing column definition from SQL. -type ColumnDef struct { - node - - Name *ColumnName - Tp *types.FieldType - Options []*ColumnOption -} - -// Accept implements Node Accept interface. -func (n *ColumnDef) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*ColumnDef) - node, ok := n.Name.Accept(v) - if !ok { - return n, false - } - n.Name = node.(*ColumnName) - for i, val := range n.Options { - node, ok := val.Accept(v) - if !ok { - return n, false - } - n.Options[i] = node.(*ColumnOption) - } - return v.Leave(n) -} - -// CreateTableStmt is a statement to create a table. -// See https://dev.mysql.com/doc/refman/5.7/en/create-table.html -type CreateTableStmt struct { - ddlNode - - IfNotExists bool - Table *TableName - ReferTable *TableName - Cols []*ColumnDef - Constraints []*Constraint - Options []*TableOption - Partition *PartitionOptions - OnDuplicate OnDuplicateCreateTableSelectType - Select ResultSetNode -} - -// Accept implements Node Accept interface. -func (n *CreateTableStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*CreateTableStmt) - node, ok := n.Table.Accept(v) - if !ok { - return n, false - } - n.Table = node.(*TableName) - if n.ReferTable != nil { - node, ok = n.ReferTable.Accept(v) - if !ok { - return n, false - } - n.ReferTable = node.(*TableName) - } - for i, val := range n.Cols { - node, ok = val.Accept(v) - if !ok { - return n, false - } - n.Cols[i] = node.(*ColumnDef) - } - for i, val := range n.Constraints { - node, ok = val.Accept(v) - if !ok { - return n, false - } - n.Constraints[i] = node.(*Constraint) - } - if n.Select != nil { - node, ok := n.Select.Accept(v) - if !ok { - return n, false - } - n.Select = node.(ResultSetNode) - } - - return v.Leave(n) -} - -// DropTableStmt is a statement to drop one or more tables. -// See https://dev.mysql.com/doc/refman/5.7/en/drop-table.html -type DropTableStmt struct { - ddlNode - - IfExists bool - Tables []*TableName -} - -// Accept implements Node Accept interface. -func (n *DropTableStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*DropTableStmt) - for i, val := range n.Tables { - node, ok := val.Accept(v) - if !ok { - return n, false - } - n.Tables[i] = node.(*TableName) - } - return v.Leave(n) -} - -// RenameTableStmt is a statement to rename a table. -// See http://dev.mysql.com/doc/refman/5.7/en/rename-table.html -type RenameTableStmt struct { - ddlNode - - OldTable *TableName - NewTable *TableName - - // TableToTables is only useful for syncer which depends heavily on tidb parser to do some dirty work for now. - // TODO: Refactor this when you are going to add full support for multiple schema changes. - TableToTables []*TableToTable -} - -// Accept implements Node Accept interface. -func (n *RenameTableStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*RenameTableStmt) - node, ok := n.OldTable.Accept(v) - if !ok { - return n, false - } - n.OldTable = node.(*TableName) - node, ok = n.NewTable.Accept(v) - if !ok { - return n, false - } - n.NewTable = node.(*TableName) - - for i, t := range n.TableToTables { - node, ok := t.Accept(v) - if !ok { - return n, false - } - n.TableToTables[i] = node.(*TableToTable) - } - - return v.Leave(n) -} - -// TableToTable represents renaming old table to new table used in RenameTableStmt. -type TableToTable struct { - node - OldTable *TableName - NewTable *TableName -} - -// Accept implements Node Accept interface. -func (n *TableToTable) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*TableToTable) - node, ok := n.OldTable.Accept(v) - if !ok { - return n, false - } - n.OldTable = node.(*TableName) - node, ok = n.NewTable.Accept(v) - if !ok { - return n, false - } - n.NewTable = node.(*TableName) - return v.Leave(n) -} - -// CreateViewStmt is a statement to create a View. -// See https://dev.mysql.com/doc/refman/5.7/en/create-view.html -type CreateViewStmt struct { - ddlNode - - OrReplace bool - ViewName *TableName - Cols []model.CIStr - Select StmtNode -} - -// Accept implements Node Accept interface. -func (n *CreateViewStmt) Accept(v Visitor) (Node, bool) { - // TODO: implement the details. - return n, true -} - -// CreateIndexStmt is a statement to create an index. -// See https://dev.mysql.com/doc/refman/5.7/en/create-index.html -type CreateIndexStmt struct { - ddlNode - - IndexName string - Table *TableName - Unique bool - IndexColNames []*IndexColName - IndexOption *IndexOption -} - -// Accept implements Node Accept interface. -func (n *CreateIndexStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*CreateIndexStmt) - node, ok := n.Table.Accept(v) - if !ok { - return n, false - } - n.Table = node.(*TableName) - for i, val := range n.IndexColNames { - node, ok = val.Accept(v) - if !ok { - return n, false - } - n.IndexColNames[i] = node.(*IndexColName) - } - if n.IndexOption != nil { - node, ok := n.IndexOption.Accept(v) - if !ok { - return n, false - } - n.IndexOption = node.(*IndexOption) - } - return v.Leave(n) -} - -// DropIndexStmt is a statement to drop the index. -// See https://dev.mysql.com/doc/refman/5.7/en/drop-index.html -type DropIndexStmt struct { - ddlNode - - IfExists bool - IndexName string - Table *TableName -} - -// Accept implements Node Accept interface. -func (n *DropIndexStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*DropIndexStmt) - node, ok := n.Table.Accept(v) - if !ok { - return n, false - } - n.Table = node.(*TableName) - return v.Leave(n) -} - -// TableOptionType is the type for TableOption -type TableOptionType int - -// TableOption types. -const ( - TableOptionNone TableOptionType = iota - TableOptionEngine - TableOptionCharset - TableOptionCollate - TableOptionAutoIncrement - TableOptionComment - TableOptionAvgRowLength - TableOptionCheckSum - TableOptionCompression - TableOptionConnection - TableOptionPassword - TableOptionKeyBlockSize - TableOptionMaxRows - TableOptionMinRows - TableOptionDelayKeyWrite - TableOptionRowFormat - TableOptionStatsPersistent - TableOptionShardRowID - TableOptionPackKeys -) - -// RowFormat types -const ( - RowFormatDefault uint64 = iota + 1 - RowFormatDynamic - RowFormatFixed - RowFormatCompressed - RowFormatRedundant - RowFormatCompact -) - -// OnDuplicateCreateTableSelectType is the option that handle unique key values in 'CREATE TABLE ... SELECT'. -// See https://dev.mysql.com/doc/refman/5.7/en/create-table-select.html -type OnDuplicateCreateTableSelectType int - -// OnDuplicateCreateTableSelect types -const ( - OnDuplicateCreateTableSelectError OnDuplicateCreateTableSelectType = iota - OnDuplicateCreateTableSelectIgnore - OnDuplicateCreateTableSelectReplace -) - -// TableOption is used for parsing table option from SQL. -type TableOption struct { - Tp TableOptionType - StrValue string - UintValue uint64 - // Skipped indicate the TableOption can be skipped. - Skipped bool -} - -// ColumnPositionType is the type for ColumnPosition. -type ColumnPositionType int - -// ColumnPosition Types -const ( - ColumnPositionNone ColumnPositionType = iota - ColumnPositionFirst - ColumnPositionAfter -) - -// ColumnPosition represent the position of the newly added column -type ColumnPosition struct { - node - // Tp is either ColumnPositionNone, ColumnPositionFirst or ColumnPositionAfter. - Tp ColumnPositionType - // RelativeColumn is the column the newly added column after if type is ColumnPositionAfter - RelativeColumn *ColumnName -} - -// Accept implements Node Accept interface. -func (n *ColumnPosition) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*ColumnPosition) - if n.RelativeColumn != nil { - node, ok := n.RelativeColumn.Accept(v) - if !ok { - return n, false - } - n.RelativeColumn = node.(*ColumnName) - } - return v.Leave(n) -} - -// AlterTableType is the type for AlterTableSpec. -type AlterTableType int - -// AlterTable types. -const ( - AlterTableOption AlterTableType = iota + 1 - AlterTableAddColumns - AlterTableAddConstraint - AlterTableDropColumn - AlterTableDropPrimaryKey - AlterTableDropIndex - AlterTableDropForeignKey - AlterTableModifyColumn - AlterTableChangeColumn - AlterTableRenameTable - AlterTableAlterColumn - AlterTableLock - AlterTableAlgorithm - AlterTableRenameIndex - AlterTableForce - AlterTableAddPartitions - AlterTableDropPartition - -// TODO: Add more actions -) - -// LockType is the type for AlterTableSpec. -// See https://dev.mysql.com/doc/refman/5.7/en/alter-table.html#alter-table-concurrency -type LockType byte - -// Lock Types. -const ( - LockTypeNone LockType = iota + 1 - LockTypeDefault - LockTypeShared - LockTypeExclusive -) - -// AlterTableSpec represents alter table specification. -type AlterTableSpec struct { - node - - Tp AlterTableType - Name string - Constraint *Constraint - Options []*TableOption - NewTable *TableName - NewColumns []*ColumnDef - OldColumnName *ColumnName - Position *ColumnPosition - LockType LockType - Comment string - FromKey model.CIStr - ToKey model.CIStr - PartDefinitions []*PartitionDefinition -} - -// Accept implements Node Accept interface. -func (n *AlterTableSpec) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*AlterTableSpec) - if n.Constraint != nil { - node, ok := n.Constraint.Accept(v) - if !ok { - return n, false - } - n.Constraint = node.(*Constraint) - } - if n.NewTable != nil { - node, ok := n.NewTable.Accept(v) - if !ok { - return n, false - } - n.NewTable = node.(*TableName) - } - for _, col := range n.NewColumns { - node, ok := col.Accept(v) - if !ok { - return n, false - } - col = node.(*ColumnDef) - } - if n.OldColumnName != nil { - node, ok := n.OldColumnName.Accept(v) - if !ok { - return n, false - } - n.OldColumnName = node.(*ColumnName) - } - if n.Position != nil { - node, ok := n.Position.Accept(v) - if !ok { - return n, false - } - n.Position = node.(*ColumnPosition) - } - return v.Leave(n) -} - -// AlterTableStmt is a statement to change the structure of a table. -// See https://dev.mysql.com/doc/refman/5.7/en/alter-table.html -type AlterTableStmt struct { - ddlNode - - Table *TableName - Specs []*AlterTableSpec -} - -// Accept implements Node Accept interface. -func (n *AlterTableStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*AlterTableStmt) - node, ok := n.Table.Accept(v) - if !ok { - return n, false - } - n.Table = node.(*TableName) - for i, val := range n.Specs { - node, ok = val.Accept(v) - if !ok { - return n, false - } - n.Specs[i] = node.(*AlterTableSpec) - } - return v.Leave(n) -} - -// TruncateTableStmt is a statement to empty a table completely. -// See https://dev.mysql.com/doc/refman/5.7/en/truncate-table.html -type TruncateTableStmt struct { - ddlNode - - Table *TableName -} - -// Accept implements Node Accept interface. -func (n *TruncateTableStmt) Accept(v Visitor) (Node, bool) { - newNode, skipChildren := v.Enter(n) - if skipChildren { - return v.Leave(newNode) - } - n = newNode.(*TruncateTableStmt) - node, ok := n.Table.Accept(v) - if !ok { - return n, false - } - n.Table = node.(*TableName) - return v.Leave(n) -} - -// PartitionDefinition defines a single partition. -type PartitionDefinition struct { - Name model.CIStr - LessThan []ExprNode - MaxValue bool - Comment string -} - -// PartitionOptions specifies the partition options. -type PartitionOptions struct { - Tp model.PartitionType - Expr ExprNode - ColumnNames []*ColumnName - Definitions []*PartitionDefinition -} diff --git a/model/ddl.go b/model/ddl.go deleted file mode 100644 index 93d1fa8e69602..0000000000000 --- a/model/ddl.go +++ /dev/null @@ -1,385 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package model - -import ( - "encoding/json" - "fmt" - "math" - "sync" - "time" - - "github.com/pingcap/tidb/terror" - "github.com/pkg/errors" -) - -// ActionType is the type for DDL action. -type ActionType byte - -// List DDL actions. -const ( - ActionNone ActionType = 0 - ActionCreateSchema ActionType = 1 - ActionDropSchema ActionType = 2 - ActionCreateTable ActionType = 3 - ActionDropTable ActionType = 4 - ActionAddColumn ActionType = 5 - ActionDropColumn ActionType = 6 - ActionAddIndex ActionType = 7 - ActionDropIndex ActionType = 8 - ActionAddForeignKey ActionType = 9 - ActionDropForeignKey ActionType = 10 - ActionTruncateTable ActionType = 11 - ActionModifyColumn ActionType = 12 - ActionRebaseAutoID ActionType = 13 - ActionRenameTable ActionType = 14 - ActionSetDefaultValue ActionType = 15 - ActionShardRowID ActionType = 16 - ActionModifyTableComment ActionType = 17 - ActionRenameIndex ActionType = 18 - ActionAddTablePartition ActionType = 19 - ActionDropTablePartition ActionType = 20 - ActionModifyTableCharsetAndCollate ActionType = 21 -) - -// AddIndexStr is a string related to the operation of "add index". -const AddIndexStr = "add index" - -var actionMap = map[ActionType]string{ - ActionCreateSchema: "create schema", - ActionDropSchema: "drop schema", - ActionCreateTable: "create table", - ActionDropTable: "drop table", - ActionAddColumn: "add column", - ActionDropColumn: "drop column", - ActionAddIndex: AddIndexStr, - ActionDropIndex: "drop index", - ActionAddForeignKey: "add foreign key", - ActionDropForeignKey: "drop foreign key", - ActionTruncateTable: "truncate table", - ActionModifyColumn: "modify column", - ActionRebaseAutoID: "rebase auto_increment ID", - ActionRenameTable: "rename table", - ActionSetDefaultValue: "set default value", - ActionShardRowID: "shard row ID", - ActionModifyTableComment: "modify table comment", - ActionRenameIndex: "rename index", - ActionAddTablePartition: "add partition", - ActionDropTablePartition: "drop table partition", -} - -// String return current ddl action in string -func (action ActionType) String() string { - if v, ok := actionMap[action]; ok { - return v - } - return "none" -} - -// HistoryInfo is used for binlog. -type HistoryInfo struct { - SchemaVersion int64 - DBInfo *DBInfo - TableInfo *TableInfo - FinishedTS uint64 -} - -// AddDBInfo adds schema version and schema information that are used for binlog. -// dbInfo is added in the following operations: create database, drop database. -func (h *HistoryInfo) AddDBInfo(schemaVer int64, dbInfo *DBInfo) { - h.SchemaVersion = schemaVer - h.DBInfo = dbInfo -} - -// AddTableInfo adds schema version and table information that are used for binlog. -// tblInfo is added except for the following operations: create database, drop database. -func (h *HistoryInfo) AddTableInfo(schemaVer int64, tblInfo *TableInfo) { - h.SchemaVersion = schemaVer - h.TableInfo = tblInfo -} - -// Clean cleans history information. -func (h *HistoryInfo) Clean() { - h.SchemaVersion = 0 - h.DBInfo = nil - h.TableInfo = nil -} - -// DDLReorgMeta is meta info of DDL reorganization. -type DDLReorgMeta struct { - // EndHandle is the last handle of the adding indices table. - // We should only backfill indices in the range [startHandle, EndHandle]. - EndHandle int64 `json:"end_handle"` -} - -// NewDDLReorgMeta new a DDLReorgMeta. -func NewDDLReorgMeta() *DDLReorgMeta { - return &DDLReorgMeta{ - EndHandle: math.MaxInt64, - } -} - -// Job is for a DDL operation. -type Job struct { - ID int64 `json:"id"` - Type ActionType `json:"type"` - SchemaID int64 `json:"schema_id"` - TableID int64 `json:"table_id"` - State JobState `json:"state"` - Error *terror.Error `json:"err"` - // ErrorCount will be increased, every time we meet an error when running job. - ErrorCount int64 `json:"err_count"` - // RowCount means the number of rows that are processed. - RowCount int64 `json:"row_count"` - Mu sync.Mutex `json:"-"` - Args []interface{} `json:"-"` - // RawArgs : We must use json raw message to delay parsing special args. - RawArgs json.RawMessage `json:"raw_args"` - SchemaState SchemaState `json:"schema_state"` - // SnapshotVer means snapshot version for this job. - SnapshotVer uint64 `json:"snapshot_ver"` - // StartTS uses timestamp allocated by TSO. - // Now it's the TS when we put the job to TiKV queue. - StartTS uint64 `json:"start_ts"` - // DependencyID is the job's ID that the current job depends on. - DependencyID int64 `json:"dependency_id"` - // Query string of the ddl job. - Query string `json:"query"` - BinlogInfo *HistoryInfo `json:"binlog"` - - // Version indicates the DDL job version. For old jobs, it will be 0. - Version int64 `json:"version"` - - // ReorgMeta is meta info of ddl reorganization. - // This field is depreciated. - ReorgMeta *DDLReorgMeta `json:"reorg_meta"` - - // Priority is only used to set the operation priority of adding indices. - Priority int `json:"priority"` -} - -// FinishTableJob is called when a job is finished. -// It updates the job's state information and adds tblInfo to the binlog. -func (job *Job) FinishTableJob(jobState JobState, schemaState SchemaState, ver int64, tblInfo *TableInfo) { - job.State = jobState - job.SchemaState = schemaState - job.BinlogInfo.AddTableInfo(ver, tblInfo) -} - -// FinishDBJob is called when a job is finished. -// It updates the job's state information and adds dbInfo the binlog. -func (job *Job) FinishDBJob(jobState JobState, schemaState SchemaState, ver int64, dbInfo *DBInfo) { - job.State = jobState - job.SchemaState = schemaState - job.BinlogInfo.AddDBInfo(ver, dbInfo) -} - -// TSConvert2Time converts timestamp to time. -func TSConvert2Time(ts uint64) time.Time { - t := int64(ts >> 18) // 18 is for the logical time. - return time.Unix(t/1e3, (t%1e3)*1e6) -} - -// SetRowCount sets the number of rows. Make sure it can pass `make race`. -func (job *Job) SetRowCount(count int64) { - job.Mu.Lock() - defer job.Mu.Unlock() - - job.RowCount = count -} - -// GetRowCount gets the number of rows. Make sure it can pass `make race`. -func (job *Job) GetRowCount() int64 { - job.Mu.Lock() - defer job.Mu.Unlock() - - return job.RowCount -} - -// Encode encodes job with json format. -// updateRawArgs is used to determine whether to update the raw args. -func (job *Job) Encode(updateRawArgs bool) ([]byte, error) { - var err error - if updateRawArgs { - job.RawArgs, err = json.Marshal(job.Args) - if err != nil { - return nil, errors.Trace(err) - } - } - - var b []byte - job.Mu.Lock() - defer job.Mu.Unlock() - b, err = json.Marshal(job) - - return b, errors.Trace(err) -} - -// Decode decodes job from the json buffer, we must use DecodeArgs later to -// decode special args for this job. -func (job *Job) Decode(b []byte) error { - err := json.Unmarshal(b, job) - return errors.Trace(err) -} - -// DecodeArgs decodes job args. -func (job *Job) DecodeArgs(args ...interface{}) error { - job.Args = args - err := json.Unmarshal(job.RawArgs, &job.Args) - return errors.Trace(err) -} - -// String implements fmt.Stringer interface. -func (job *Job) String() string { - rowCount := job.GetRowCount() - return fmt.Sprintf("ID:%d, Type:%s, State:%s, SchemaState:%s, SchemaID:%d, TableID:%d, RowCount:%d, ArgLen:%d, start time: %v, Err:%v, ErrCount:%d, SnapshotVersion:%v", - job.ID, job.Type, job.State, job.SchemaState, job.SchemaID, job.TableID, rowCount, len(job.Args), TSConvert2Time(job.StartTS), job.Error, job.ErrorCount, job.SnapshotVer) -} - -func (job *Job) hasDependentSchema(other *Job) (bool, error) { - if other.Type == ActionDropSchema || other.Type == ActionCreateSchema { - if other.SchemaID == job.SchemaID { - return true, nil - } - if job.Type == ActionRenameTable { - var oldSchemaID int64 - if err := job.DecodeArgs(&oldSchemaID); err != nil { - return false, errors.Trace(err) - } - if other.SchemaID == oldSchemaID { - return true, nil - } - } - } - return false, nil -} - -// IsDependentOn returns whether the job depends on "other". -// How to check the job depends on "other"? -// 1. The two jobs handle the same database when one of the two jobs is an ActionDropSchema or ActionCreateSchema type. -// 2. Or the two jobs handle the same table. -func (job *Job) IsDependentOn(other *Job) (bool, error) { - isDependent, err := job.hasDependentSchema(other) - if err != nil || isDependent { - return isDependent, errors.Trace(err) - } - isDependent, err = other.hasDependentSchema(job) - if err != nil || isDependent { - return isDependent, errors.Trace(err) - } - - // TODO: If a job is ActionRenameTable, we need to check table name. - if other.TableID == job.TableID { - return true, nil - } - return false, nil -} - -// IsFinished returns whether job is finished or not. -// If the job state is Done or Cancelled, it is finished. -func (job *Job) IsFinished() bool { - return job.State == JobStateDone || job.State == JobStateRollbackDone || job.State == JobStateCancelled -} - -// IsCancelled returns whether the job is cancelled or not. -func (job *Job) IsCancelled() bool { - return job.State == JobStateCancelled -} - -// IsRollbackDone returns whether the job is rolled back or not. -func (job *Job) IsRollbackDone() bool { - return job.State == JobStateRollbackDone -} - -// IsRollingback returns whether the job is rolling back or not. -func (job *Job) IsRollingback() bool { - return job.State == JobStateRollingback -} - -// IsCancelling returns whether the job is cancelling or not. -func (job *Job) IsCancelling() bool { - return job.State == JobStateCancelling -} - -// IsSynced returns whether the DDL modification is synced among all TiDB servers. -func (job *Job) IsSynced() bool { - return job.State == JobStateSynced -} - -// IsDone returns whether job is done. -func (job *Job) IsDone() bool { - return job.State == JobStateDone -} - -// IsRunning returns whether job is still running or not. -func (job *Job) IsRunning() bool { - return job.State == JobStateRunning -} - -// JobState is for job state. -type JobState byte - -// List job states. -const ( - JobStateNone JobState = 0 - JobStateRunning JobState = 1 - // When DDL encountered an unrecoverable error at reorganization state, - // some keys has been added already, we need to remove them. - // JobStateRollingback is the state to do the rolling back job. - JobStateRollingback JobState = 2 - JobStateRollbackDone JobState = 3 - JobStateDone JobState = 4 - JobStateCancelled JobState = 5 - // JobStateSynced is used to mark the information about the completion of this job - // has been synchronized to all servers. - JobStateSynced JobState = 6 - // JobStateCancelling is used to mark the DDL job is cancelled by the client, but the DDL work hasn't handle it. - JobStateCancelling JobState = 7 -) - -// String implements fmt.Stringer interface. -func (s JobState) String() string { - switch s { - case JobStateRunning: - return "running" - case JobStateRollingback: - return "rollingback" - case JobStateRollbackDone: - return "rollback done" - case JobStateDone: - return "done" - case JobStateCancelled: - return "cancelled" - case JobStateCancelling: - return "cancelling" - case JobStateSynced: - return "synced" - default: - return "none" - } -} - -// SchemaDiff contains the schema modification at a particular schema version. -// It is used to reduce schema reload cost. -type SchemaDiff struct { - Version int64 `json:"version"` - Type ActionType `json:"type"` - SchemaID int64 `json:"schema_id"` - TableID int64 `json:"table_id"` - - // OldTableID is the table ID before truncate, only used by truncate table DDL. - OldTableID int64 `json:"old_table_id"` - // OldSchemaID is the schema ID before rename table, only used by rename table DDL. - OldSchemaID int64 `json:"old_schema_id"` -} From e3fd359fe519b08bc2888badbd45317feb97b4d7 Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 12 Nov 2018 12:12:47 +0800 Subject: [PATCH 04/13] replace go module --- ddl/column_test.go | 2 +- ddl/db_change_test.go | 2 +- ddl/db_test.go | 2 +- go.mod | 2 ++ go.sum | 4 ++++ 5 files changed, 9 insertions(+), 3 deletions(-) diff --git a/ddl/column_test.go b/ddl/column_test.go index b5a1e3edb2a64..9ed50707e565d 100644 --- a/ddl/column_test.go +++ b/ddl/column_test.go @@ -915,7 +915,7 @@ func (s *testColumnSuite) TestModifyColumn(c *C) { {"int", "int unsigned", errUnsupportedModifyColumn.GenWithStackByArgs("length 10 is less than origin 11")}, {"varchar(10)", "text", nil}, {"varbinary(10)", "blob", nil}, - {"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8 to binary")}, + {"text", "blob", errUnsupportedModifyCharset.GenWithStackByArgs("charset from utf8mb4 to binary")}, {"varchar(10)", "varchar(8)", errUnsupportedModifyColumn.GenWithStackByArgs("length 8 is less than origin 10")}, {"varchar(10)", "varchar(11)", nil}, {"varchar(10) character set utf8 collate utf8_bin", "varchar(10) character set utf8", nil}, diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 58d49cfd388fb..4ccbbae213791 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -376,7 +376,7 @@ func (s *testStateChangeSuite) TestAppendEnum(c *C) { c.Assert(err.Error(), Equals, "[ddl:203]unsupported modify column the number of enum column's elements is less than the original: 2") failAlterTableSQL2 := "alter table t change c2 c2 int default 0" _, err = s.se.Execute(context.Background(), failAlterTableSQL2) - c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8 to binary") + c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8mb4 to binary") alterTableSQL := "alter table t change c2 c2 enum('N','Y','A') DEFAULT 'A'" _, err = s.se.Execute(context.Background(), alterTableSQL) c.Assert(err, IsNil) diff --git a/ddl/db_test.go b/ddl/db_test.go index 5d441589d5e12..5beb912755a49 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -26,6 +26,7 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" + "github.com/pingcap/parser/charset" "github.com/pingcap/parser/model" "github.com/pingcap/parser/mysql" tmysql "github.com/pingcap/parser/mysql" @@ -45,7 +46,6 @@ import ( "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/admin" - "github.com/pingcap/tidb/util/charset" "github.com/pingcap/tidb/util/mock" "github.com/pingcap/tidb/util/testkit" "github.com/pingcap/tidb/util/testleak" diff --git a/go.mod b/go.mod index 3906a25895fc2..8e3692a753ab5 100644 --- a/go.mod +++ b/go.mod @@ -48,3 +48,5 @@ require ( google.golang.org/grpc v1.16.0 gopkg.in/natefinch/lumberjack.v2 v2.0.0 ) + +replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181112035727-04e30ab49120 diff --git a/go.sum b/go.sum index 02271d68a0a68..be1fe030cd33c 100644 --- a/go.sum +++ b/go.sum @@ -295,6 +295,10 @@ github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= github.com/urfave/negroni v1.0.0 h1:kIimOitoypq34K7TG7DUaJ9kq/N4Ofuwi1sjz0KipXc= github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= +github.com/winkyao/parser v0.0.0-20181105094001-d4464cdde59f h1:go055dFjiXKeRhPJpSw/NZKYqkI3ak2j+mPO7gA/YK0= +github.com/winkyao/parser v0.0.0-20181105094001-d4464cdde59f/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= +github.com/winkyao/parser v0.0.0-20181112035727-04e30ab49120 h1:7aaQMMs2d67LVC9KkzyapCy5VMm2HS/k3PHD/nB0G2Q= +github.com/winkyao/parser v0.0.0-20181112035727-04e30ab49120/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= github.com/zimulala/parser v0.0.0-20181108115726-650caa67be64 h1:QYGszvB1bmVsjvH6Id0AAnYXK9KUzSlbD8sbfaSf+Rw= From 28694971806a4b9eb39e0372df91e5ec1ba69029 Mon Sep 17 00:00:00 2001 From: winkyao Date: Mon, 3 Dec 2018 19:45:19 +0800 Subject: [PATCH 05/13] fix go mod tidy --- go.sum | 8 -------- 1 file changed, 8 deletions(-) diff --git a/go.sum b/go.sum index 01168b36546d6..827f8594de1c8 100644 --- a/go.sum +++ b/go.sum @@ -160,14 +160,6 @@ github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf h1:BLcwkDfQ8QPXNXB github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4QUw6lhHsVymzwSDJOZcE0s2X8S20= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= -github.com/urfave/negroni v0.3.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/urfave/negroni v1.0.0 h1:kIimOitoypq34K7TG7DUaJ9kq/N4Ofuwi1sjz0KipXc= -github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= -github.com/winkyao/parser v0.0.0-20181105094001-d4464cdde59f h1:go055dFjiXKeRhPJpSw/NZKYqkI3ak2j+mPO7gA/YK0= -github.com/winkyao/parser v0.0.0-20181105094001-d4464cdde59f/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= -github.com/winkyao/parser v0.0.0-20181112035727-04e30ab49120 h1:7aaQMMs2d67LVC9KkzyapCy5VMm2HS/k3PHD/nB0G2Q= -github.com/winkyao/parser v0.0.0-20181112035727-04e30ab49120/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= -github.com/winkyao/parser v0.0.0-20181203021712-d8af28b80152 h1:PPK1alQj9NdRKVY9N9fhFpfQSDhgQu3ga/uXZGJ8VJw= github.com/winkyao/parser v0.0.0-20181203021712-d8af28b80152/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= From e6df64f0149988ab1d66ae7cd9e1d792e6c0bf6e Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 4 Dec 2018 16:44:44 +0800 Subject: [PATCH 06/13] fix ci --- ddl/ddl.go | 1 + 1 file changed, 1 insertion(+) diff --git a/ddl/ddl.go b/ddl/ddl.go index 2bdfc3cb36e46..4d04ba92881c9 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -588,6 +588,7 @@ const ( codeUnsupportedShardRowIDBits = 207 codeUnsupportedAddPartition = 208 codeUnsupportedCoalescePartition = 209 + codeUnsupportedModifyCharset = 210 codeFileNotFound = 1017 codeErrorOnRename = 1025 From aeba21e999278c6bcd814ffa082c74880c572a6c Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 4 Dec 2018 17:08:56 +0800 Subject: [PATCH 07/13] fix ci --- go.mod | 2 +- go.sum | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 76c3ec10efb50..53bb144de4b2f 100644 --- a/go.mod +++ b/go.mod @@ -85,4 +85,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181203021712-d8af28b80152 +replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181204090725-db135ad3be2b diff --git a/go.sum b/go.sum index 348c93487aadd..54589e0280a72 100644 --- a/go.sum +++ b/go.sum @@ -160,7 +160,7 @@ github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf h1:BLcwkDfQ8QPXNXB github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4QUw6lhHsVymzwSDJOZcE0s2X8S20= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= -github.com/winkyao/parser v0.0.0-20181203021712-d8af28b80152/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= +github.com/winkyao/parser v0.0.0-20181204090725-db135ad3be2b/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= From d4a9027ecf59c1d0df66e8211022a09fe07dea5b Mon Sep 17 00:00:00 2001 From: winkyao Date: Tue, 4 Dec 2018 17:20:49 +0800 Subject: [PATCH 08/13] fix ci --- ddl/db_change_test.go | 2 +- ddl/db_test.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/ddl/db_change_test.go b/ddl/db_change_test.go index 4ccbbae213791..c4e34e80cec79 100644 --- a/ddl/db_change_test.go +++ b/ddl/db_change_test.go @@ -376,7 +376,7 @@ func (s *testStateChangeSuite) TestAppendEnum(c *C) { c.Assert(err.Error(), Equals, "[ddl:203]unsupported modify column the number of enum column's elements is less than the original: 2") failAlterTableSQL2 := "alter table t change c2 c2 int default 0" _, err = s.se.Execute(context.Background(), failAlterTableSQL2) - c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8mb4 to binary") + c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify charset from utf8mb4 to binary") alterTableSQL := "alter table t change c2 c2 enum('N','Y','A') DEFAULT 'A'" _, err = s.se.Execute(context.Background(), alterTableSQL) c.Assert(err, IsNil) diff --git a/ddl/db_test.go b/ddl/db_test.go index 856ea6679639c..9c2e22e7a148a 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -3758,7 +3758,7 @@ func (s *testDBSuite) TestAlterTableCharset(c *C) { rs.Close() } - c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8mb4 to utf8") + c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify charset from utf8mb4 to utf8") } func (s *testDBSuite) TestAlterColumnCharset(c *C) { @@ -3812,7 +3812,7 @@ func (s *testDBSuite) TestAlterColumnCharset(c *C) { if rs != nil { rs.Close() } - c.Assert(err.Error(), Equals, "[ddl:208]unsupported modify charset from utf8mb4 to utf8") + c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify charset from utf8mb4 to utf8") } func (s *testDBSuite) TestDropSchemaWithPartitionTable(c *C) { From b15d3b191700c3919f9eb06027bd506007d361c9 Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 6 Dec 2018 21:44:14 +0800 Subject: [PATCH 09/13] address comments --- ddl/ddl_api.go | 29 ++++++++++++++++------------- go.mod | 2 +- go.sum | 3 ++- 3 files changed, 19 insertions(+), 15 deletions(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 1402d128f1c07..cef7b9698565f 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1092,25 +1092,31 @@ func isIgnorableSpec(tp ast.AlterTableType) bool { } // getCharsetAndCollateInTableOption will iterate the charset and collate in the options, -// and returns the last charset and collate in options. +// and returns the last charset and collate in options. If there is no charset in the options, +// the returns charset will be "", the same as collate. func getCharsetAndCollateInTableOption(startIdx int, options []*ast.TableOption) (charset, collate string) { + charsets := make([]string, len(options)) + collates := make([]string, len(options)) for i := startIdx; i < len(options); i++ { opt := options[i] // we set the charset to the last option. example: alter table t charset latin1 charset utf8 collate utf8_bin; // the charset will be utf8, collate will be utf8_bin switch opt.Tp { case ast.TableOptionCharset: - charset = opt.StrValue - // this opt is handled, so we can skipped in the next iteration. - options[i].Skipped = true + charsets = append(charsets, opt.StrValue) case ast.TableOptionCollate: - collate = opt.StrValue - // this opt is handled, so we can skipped in the next iteration. - options[i].Skipped = true + collates = append(collates, opt.StrValue) } } - return charset, collate + if len(charsets) != 0 { + charset = charsets[len(charsets)-1] + } + + if len(collates) != 0 { + collate = collates[len(collates)-1] + } + return } func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.AlterTableSpec) (err error) { @@ -1177,10 +1183,6 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A err = d.RenameIndex(ctx, ident, spec) case ast.AlterTableOption: for i, opt := range spec.Options { - if opt.Skipped { - continue - } - switch opt.Tp { case ast.TableOptionShardRowID: if opt.UintValue > shardRowIDBitsMax { @@ -1938,7 +1940,8 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a } // AlterTableCharset changes the table charset and collate. -func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset string, toCollate string) error { +func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset, toCollate string) error { + // use the last one. if toCharset == "" && toCollate == "" { return errors.Errorf("toCharset and toCollate can't be empty") } diff --git a/go.mod b/go.mod index 53bb144de4b2f..235dd6f21598e 100644 --- a/go.mod +++ b/go.mod @@ -85,4 +85,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181204090725-db135ad3be2b +replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd diff --git a/go.sum b/go.sum index 54589e0280a72..d1be8d0482dab 100644 --- a/go.sum +++ b/go.sum @@ -160,7 +160,8 @@ github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf h1:BLcwkDfQ8QPXNXB github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4QUw6lhHsVymzwSDJOZcE0s2X8S20= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= -github.com/winkyao/parser v0.0.0-20181204090725-db135ad3be2b/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= +github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd h1:4pV13Aq/CiXd3gKdGAliGFAWo9FDTYUu3BFJ2K6Di5g= +github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= From b4dce09055d2d40fd93b6727767f3f420f622bda Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 6 Dec 2018 21:44:43 +0800 Subject: [PATCH 10/13] go mod tidy --- go.sum | 1 - 1 file changed, 1 deletion(-) diff --git a/go.sum b/go.sum index d1be8d0482dab..5583a33ae1762 100644 --- a/go.sum +++ b/go.sum @@ -160,7 +160,6 @@ github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf h1:BLcwkDfQ8QPXNXB github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4QUw6lhHsVymzwSDJOZcE0s2X8S20= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= -github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd h1:4pV13Aq/CiXd3gKdGAliGFAWo9FDTYUu3BFJ2K6Di5g= github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= From c1e004ad1984ed5d8763403b90097cc0266b19e6 Mon Sep 17 00:00:00 2001 From: winkyao Date: Thu, 6 Dec 2018 21:48:44 +0800 Subject: [PATCH 11/13] handle once --- ddl/ddl_api.go | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index cef7b9698565f..8286611e27957 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1136,6 +1136,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } for _, spec := range validSpecs { + var handledCharsetOrCollate bool switch spec.Tp { case ast.AlterTableAddColumns: if len(spec.NewColumns) != 1 { @@ -1195,8 +1196,14 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A spec.Comment = opt.StrValue err = d.AlterTableComment(ctx, ident, spec) case ast.TableOptionCharset, ast.TableOptionCollate: + // getCharsetAndCollateInTableOption will get the last charset and collate in the options, + // so it should be handled only once. + if handledCharsetOrCollate { + continue + } toCharset, toCollate := getCharsetAndCollateInTableOption(i, spec.Options) err = d.AlterTableCharsetAndCollate(ctx, ident, toCharset, toCollate) + handledCharsetOrCollate = true } if err != nil { From 36681fa823285333020eb0e391689d03c82344bd Mon Sep 17 00:00:00 2001 From: winkyao Date: Fri, 7 Dec 2018 13:35:23 +0800 Subject: [PATCH 12/13] update parser to fix ci --- go.mod | 2 +- go.sum | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go.mod b/go.mod index 66b20f7fb0144..a97df5cc881c3 100644 --- a/go.mod +++ b/go.mod @@ -86,4 +86,4 @@ require ( sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) -replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd +replace github.com/pingcap/parser => github.com/winkyao/parser v0.0.0-20181207053352-37ef416fe2a0 diff --git a/go.sum b/go.sum index a4118a38e0ad3..c191c40545c16 100644 --- a/go.sum +++ b/go.sum @@ -162,7 +162,7 @@ github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf h1:BLcwkDfQ8QPXNXB github.com/ugorji/go/codec v0.0.0-20181127175209-856da096dbdf/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d h1:ggUgChAeyge4NZ4QUw6lhHsVymzwSDJOZcE0s2X8S20= github.com/unrolled/render v0.0.0-20180914162206-b9786414de4d/go.mod h1:tu82oB5W2ykJRVioYsB+IQKcft7ryBr7w12qMBUPyXg= -github.com/winkyao/parser v0.0.0-20181206132759-921d258011cd/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= +github.com/winkyao/parser v0.0.0-20181207053352-37ef416fe2a0/go.mod h1:R7TohkGdsluhClCn0ZTwhHDjjjqoiBr6DFFAPTaFvUI= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18 h1:MPPkRncZLN9Kh4MEFmbnK4h3BD7AUmskWv2+EeZJCCs= github.com/xiang90/probing v0.0.0-20160813154853-07dd2e8dfe18/go.mod h1:UETIi67q53MR2AWcXfiuqkDkRtnGDLqkBTpCHuJHxtU= go.uber.org/atomic v1.3.2 h1:2Oa65PReHzfn29GpvgsYwloV9AVFHPDk8tYxt2c2tr4= From a114ea4c59e95865af80d839f2be9e7ee892df71 Mon Sep 17 00:00:00 2001 From: winkyao Date: Fri, 7 Dec 2018 23:42:42 +0800 Subject: [PATCH 13/13] address comment --- ddl/ddl_api.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 1afa2472e20e0..721d6632263cc 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1950,7 +1950,7 @@ func (d *ddl) AlterTableComment(ctx sessionctx.Context, ident ast.Ident, spec *a func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Ident, toCharset, toCollate string) error { // use the last one. if toCharset == "" && toCollate == "" { - return errors.Errorf("toCharset and toCollate can't be empty") + return ErrUnknownCharacterSet.GenWithStackByArgs(toCharset) } is := d.infoHandle.Get()