diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 3a3ee36eb6481..7ebd664e4627a 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" "fmt" + "strings" . "github.com/pingcap/check" "github.com/pingcap/errors" @@ -517,3 +518,108 @@ func (s *testIntegrationSuite) TestIgnoreColumnUTF8Charset(c *C) { " `b` varchar(50) CHARSET ascii COLLATE ascii_bin DEFAULT NULL\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin")) } + +func (s *testIntegrationSuite) TestChangingDBCharset(c *C) { + tk := testkit.NewTestKit(c, s.store) + + tk.MustExec("DROP DATABASE IF EXISTS alterdb1") + tk.MustExec("CREATE DATABASE alterdb1 CHARSET=utf8 COLLATE=utf8_unicode_ci") + + // No default DB errors. + noDBFailedCases := []struct { + stmt string + errMsg string + }{ + { + "ALTER DATABASE CHARACTER SET = 'utf8'", + "[planner:1046]No database selected", + }, + { + "ALTER SCHEMA `` CHARACTER SET = 'utf8'", + "[ddl:1102]Incorrect database name ''", + }, + } + for _, fc := range noDBFailedCases { + _, err := tk.Exec(fc.stmt) + c.Assert(err.Error(), Equals, fc.errMsg, Commentf("%v", fc.stmt)) + } + + verifyDBCharsetAndCollate := func(dbName, chs string, coll string) { + // check `SHOW CREATE SCHEMA`. + r := tk.MustQuery("SHOW CREATE SCHEMA " + dbName).Rows()[0][1].(string) + c.Assert(strings.Contains(r, "CHARACTER SET "+chs), IsTrue) + + template := `SELECT + DEFAULT_CHARACTER_SET_NAME, + DEFAULT_COLLATION_NAME + FROM INFORMATION_SCHEMA.SCHEMATA + WHERE SCHEMA_NAME = '%s'` + sql := fmt.Sprintf(template, dbName) + tk.MustQuery(sql).Check(testkit.Rows(fmt.Sprintf("%s %s", chs, coll))) + + dom := domain.GetDomain(tk.Se.(sessionctx.Context)) + // Make sure the table schema is the new schema. + err := dom.Reload() + c.Assert(err, IsNil) + dbInfo, ok := dom.InfoSchema().SchemaByName(model.NewCIStr(dbName)) + c.Assert(ok, Equals, true) + c.Assert(dbInfo.Charset, Equals, chs) + c.Assert(dbInfo.Collate, Equals, coll) + } + + tk.MustExec("ALTER SCHEMA alterdb1 COLLATE = utf8mb4_general_ci") + verifyDBCharsetAndCollate("alterdb1", "utf8mb4", "utf8mb4_general_ci") + + tk.MustExec("DROP DATABASE IF EXISTS alterdb2") + tk.MustExec("CREATE DATABASE alterdb2 CHARSET=utf8 COLLATE=utf8_unicode_ci") + tk.MustExec("USE alterdb2") + + failedCases := []struct { + stmt string + errMsg string + }{ + { + "ALTER SCHEMA `` CHARACTER SET = 'utf8'", + "[ddl:1102]Incorrect database name ''", + }, + { + "ALTER DATABASE CHARACTER SET = ''", + "[parser:1115]Unknown character set: ''", + }, + { + "ALTER DATABASE CHARACTER SET = 'INVALID_CHARSET'", + "[parser:1115]Unknown character set: 'INVALID_CHARSET'", + }, + { + "ALTER SCHEMA COLLATE = ''", + "[ddl:1273]Unknown collation: ''", + }, + { + "ALTER DATABASE COLLATE = 'INVALID_COLLATION'", + "[ddl:1273]Unknown collation: 'INVALID_COLLATION'", + }, + { + "ALTER DATABASE CHARACTER SET = 'utf8' DEFAULT CHARSET = 'utf8mb4'", + "[ddl:1302]Conflicting declarations: 'CHARACTER SET utf8' and 'CHARACTER SET utf8mb4'", + }, + { + "ALTER SCHEMA CHARACTER SET = 'utf8' COLLATE = 'utf8mb4_bin'", + "[ddl:1302]Conflicting declarations: 'CHARACTER SET utf8' and 'CHARACTER SET utf8mb4'", + }, + { + "ALTER DATABASE COLLATE = 'utf8mb4_bin' COLLATE = 'utf8_bin'", + "[ddl:1302]Conflicting declarations: 'CHARACTER SET utf8mb4' and 'CHARACTER SET utf8'", + }, + } + + for _, fc := range failedCases { + _, err := tk.Exec(fc.stmt) + c.Assert(err.Error(), Equals, fc.errMsg, Commentf("%v", fc.stmt)) + } + + tk.MustExec("ALTER SCHEMA CHARACTER SET = 'utf8mb4'") + verifyDBCharsetAndCollate("alterdb2", "utf8mb4", "utf8mb4_bin") + + _, err := tk.Exec("ALTER SCHEMA CHARACTER SET = 'utf8mb4' COLLATE = 'utf8mb4_general_ci'") + c.Assert(err.Error(), Equals, "[ddl:210]unsupported modify collate from utf8mb4_bin to utf8mb4_general_ci") +} diff --git a/ddl/ddl.go b/ddl/ddl.go index 78d238522886f..ddbf92fe5ea4b 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -168,6 +168,8 @@ var ( ErrWrongNameForIndex = terror.ClassDDL.New(codeWrongNameForIndex, mysql.MySQLErrName[mysql.ErrWrongNameForIndex]) // ErrUnknownCharacterSet returns unknown character set. ErrUnknownCharacterSet = terror.ClassDDL.New(codeUnknownCharacterSet, "Unknown character set: '%s'") + // ErrConflictingDeclarations return conflict declarations. + ErrConflictingDeclarations = terror.ClassDDL.New(codeConflictingDeclarations, "Conflicting declarations: 'CHARACTER SET %s' and 'CHARACTER SET %s'") // ErrPrimaryCantHaveNull returns All parts of a PRIMARY KEY must be NOT NULL; if you need NULL in a key, use UNIQUE instead ErrPrimaryCantHaveNull = terror.ClassDDL.New(codePrimaryCantHaveNull, mysql.MySQLErrName[mysql.ErrPrimaryCantHaveNull]) @@ -205,6 +207,7 @@ var ( // DDL is responsible for updating schema in data store and maintaining in-memory InfoSchema cache. type DDL interface { CreateSchema(ctx sessionctx.Context, name model.CIStr, charsetInfo *ast.CharsetOpt) error + AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) error DropSchema(ctx sessionctx.Context, schema model.CIStr) error CreateTable(ctx sessionctx.Context, stmt *ast.CreateTableStmt) error CreateTableWithLike(ctx sessionctx.Context, ident, referIdent ast.Ident, ifNotExists bool) error @@ -617,6 +620,7 @@ const ( codeWrongNameForIndex = terror.ErrCode(mysql.ErrWrongNameForIndex) codeErrTooLongIndexComment = terror.ErrCode(mysql.ErrTooLongIndexComment) codeUnknownCharacterSet = terror.ErrCode(mysql.ErrUnknownCharacterSet) + codeConflictingDeclarations = terror.ErrCode(mysql.ErrConflictingDeclarations) codeCantCreateTable = terror.ErrCode(mysql.ErrCantCreateTable) codeTableMustHaveColumns = terror.ErrCode(mysql.ErrTableMustHaveColumns) codePartitionsMustBeDefined = terror.ErrCode(mysql.ErrPartitionsMustBeDefined) diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7d79e415cb349..d4e15e05d8772 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -82,6 +82,63 @@ func (d *ddl) CreateSchema(ctx sessionctx.Context, schema model.CIStr, charsetIn err = d.callHookOnChanged(err) return errors.Trace(err) } +func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (err error) { + // Resolve target charset and collation from options. + var toCharset, toCollate string + for _, val := range stmt.Options { + switch val.Tp { + case ast.DatabaseOptionCharset: + if toCharset == "" { + toCharset = val.Value + } else if toCharset != val.Value { + return ErrConflictingDeclarations.GenWithStackByArgs(toCharset, val.Value) + } + case ast.DatabaseOptionCollate: + info, err := charset.GetCollationByName(val.Value) + if err != nil { + return errors.Trace(err) + } + if toCharset == "" { + toCharset = info.CharsetName + } else if toCharset != info.CharsetName { + return ErrConflictingDeclarations.GenWithStackByArgs(toCharset, info.CharsetName) + } + toCollate = info.Name + } + } + if toCollate == "" { + if toCollate, err = charset.GetDefaultCollation(toCharset); err != nil { + return errors.Trace(err) + } + } + + // Check if need to change charset/collation. + dbName := model.NewCIStr(stmt.Name) + is := d.GetInformationSchema(ctx) + dbInfo, ok := is.SchemaByName(dbName) + if !ok { + return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName.O) + } + if dbInfo.Charset == toCharset && dbInfo.Charset == toCollate { + return nil + } + + // Check the current TiDB limitations. + if err = modifiableCharsetAndCollation(toCharset, toCollate, dbInfo.Charset, dbInfo.Collate); err != nil { + return errors.Trace(err) + } + + // Do the DDL job. + job := &model.Job{ + SchemaID: dbInfo.ID, + Type: model.ActionModifySchemaCharsetAndCollate, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{toCharset, toCollate}, + } + err = d.doDDLJob(ctx, job) + err = d.callHookOnChanged(err) + return errors.Trace(err) +} func (d *ddl) DropSchema(ctx sessionctx.Context, schema model.CIStr) (err error) { is := d.GetInformationSchema(ctx) diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 5e7a40628f63c..872c3ecf6d532 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -475,6 +475,8 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, switch job.Type { case model.ActionCreateSchema: ver, err = onCreateSchema(t, job) + case model.ActionModifySchemaCharsetAndCollate: + ver, err = onModifySchemaCharsetAndCollate(t, job) case model.ActionDropSchema: ver, err = onDropSchema(t, job) case model.ActionCreateTable: diff --git a/ddl/rollingback.go b/ddl/rollingback.go index 7dd1258579c8f..64fb47ed3d1e6 100644 --- a/ddl/rollingback.go +++ b/ddl/rollingback.go @@ -270,7 +270,8 @@ func convertJob2RollbackJob(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) case model.ActionRebaseAutoID, model.ActionShardRowID, model.ActionModifyColumn, model.ActionAddForeignKey, model.ActionDropForeignKey, model.ActionRenameTable, - model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition: + model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, + model.ActionModifySchemaCharsetAndCollate: ver, err = cancelOnlyNotHandledJob(job) case model.ActionDropTable, model.ActionDropSchema: job.State = model.JobStateRollingback diff --git a/ddl/schema.go b/ddl/schema.go index 8c506b67e242b..b9c513f31abdf 100644 --- a/ddl/schema.go +++ b/ddl/schema.go @@ -134,6 +134,37 @@ func onDropSchema(t *meta.Meta, job *model.Job) (ver int64, _ error) { return ver, errors.Trace(err) } +func onModifySchemaCharsetAndCollate(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) + } + + dbInfo, err := t.GetDatabase(job.SchemaID) + if err != nil { + job.State = model.JobStateCancelled + return ver, errors.Trace(err) + } + + if dbInfo.Charset == toCharset && dbInfo.Collate == toCollate { + job.State = model.JobStateCancelled + return ver, nil + } + + dbInfo.Charset = toCharset + dbInfo.Collate = toCollate + + if err = t.UpdateDatabase(dbInfo); err != nil { + return ver, errors.Trace(err) + } + if ver, err = updateSchemaVersion(t, job); err != nil { + return ver, errors.Trace(err) + } + job.FinishDBJob(model.JobStateDone, model.StatePublic, ver, dbInfo) + return ver, nil +} + func getIDs(tables []*model.TableInfo) []int64 { ids := make([]int64, 0, len(tables)) for _, t := range tables { diff --git a/executor/ddl.go b/executor/ddl.go index fac253a5790ec..77ab2d3022a50 100644 --- a/executor/ddl.go +++ b/executor/ddl.go @@ -81,24 +81,27 @@ func (e *DDLExec) Next(ctx context.Context, chk *chunk.Chunk) (err error) { defer func() { e.ctx.GetSessionVars().StmtCtx.IsDDLJobInQueue = false }() switch x := e.stmt.(type) { - case *ast.TruncateTableStmt: - err = e.executeTruncateTable(x) + case *ast.AlterDatabaseStmt: + err = e.executeAlterDatabase(x) + case *ast.AlterTableStmt: + err = e.executeAlterTable(x) case *ast.CreateDatabaseStmt: err = e.executeCreateDatabase(x) - case *ast.CreateTableStmt: - err = e.executeCreateTable(x) case *ast.CreateIndexStmt: err = e.executeCreateIndex(x) + case *ast.CreateTableStmt: + err = e.executeCreateTable(x) case *ast.DropDatabaseStmt: err = e.executeDropDatabase(x) - case *ast.DropTableStmt: - err = e.executeDropTable(x) case *ast.DropIndexStmt: err = e.executeDropIndex(x) - case *ast.AlterTableStmt: - err = e.executeAlterTable(x) + case *ast.DropTableStmt: + err = e.executeDropTable(x) case *ast.RenameTableStmt: err = e.executeRenameTable(x) + case *ast.TruncateTableStmt: + err = e.executeTruncateTable(x) + } if err != nil { return errors.Trace(e.toErr(err)) @@ -155,6 +158,11 @@ func (e *DDLExec) executeCreateDatabase(s *ast.CreateDatabaseStmt) error { return errors.Trace(err) } +func (e *DDLExec) executeAlterDatabase(s *ast.AlterDatabaseStmt) error { + err := domain.GetDomain(e.ctx).DDL().AlterSchema(e.ctx, s) + return err +} + func (e *DDLExec) executeCreateTable(s *ast.CreateTableStmt) error { err := domain.GetDomain(e.ctx).DDL().CreateTable(e.ctx, s) return errors.Trace(err) diff --git a/go.mod b/go.mod index fc26329ab5da3..118ed5699612c 100644 --- a/go.mod +++ b/go.mod @@ -48,7 +48,7 @@ require ( github.com/pingcap/goleveldb v0.0.0-20171020084629-8d44bfdf1030 github.com/pingcap/kvproto v0.0.0-20190226063853-f6c0b7ffff11 github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 - github.com/pingcap/parser v0.0.0-20190513135550-b80386add845 + github.com/pingcap/parser v0.0.0-20190516020405-361d2d4f774d github.com/pingcap/pd v2.1.0-rc.4+incompatible github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible github.com/pingcap/tipb v0.0.0-20180910045846-371b48b15d93 diff --git a/go.sum b/go.sum index d73f3013846f3..f08a8e93ef1b7 100644 --- a/go.sum +++ b/go.sum @@ -101,8 +101,8 @@ github.com/pingcap/kvproto v0.0.0-20190226063853-f6c0b7ffff11 h1:e81flSfRbbMW5RU github.com/pingcap/kvproto v0.0.0-20190226063853-f6c0b7ffff11/go.mod h1:0gwbe1F2iBIjuQ9AH0DbQhL+Dpr5GofU8fgYyXk+ykk= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ= github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw= -github.com/pingcap/parser v0.0.0-20190513135550-b80386add845 h1:xOe4TrirqX8t3ZlBjJzBqzw04XIrSvhgYGdTMIvKB/0= -github.com/pingcap/parser v0.0.0-20190513135550-b80386add845/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= +github.com/pingcap/parser v0.0.0-20190516020405-361d2d4f774d h1:KfAKc8SCBXKtvuyqrncd2gpBSzudNJEtKEHR5lqu1BM= +github.com/pingcap/parser v0.0.0-20190516020405-361d2d4f774d/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA= github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE= github.com/pingcap/pd v2.1.0-rc.4+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E= github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible h1:e9Gi/LP9181HT3gBfSOeSBA+5JfemuE4aEAhqNgoE4k= diff --git a/infoschema/builder.go b/infoschema/builder.go index 21d5ac881904d..4276d9ae6eb4a 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -44,6 +44,8 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro } else if diff.Type == model.ActionDropSchema { tblIDs := b.applyDropSchema(diff.SchemaID) return tblIDs, nil + } else if diff.Type == model.ActionModifySchemaCharsetAndCollate { + return nil, b.applyModifySchemaCharsetAndCollate(m, diff) } roDBInfo, ok := b.is.SchemaByID(diff.SchemaID) @@ -128,6 +130,23 @@ func (b *Builder) applyCreateSchema(m *meta.Meta, diff *model.SchemaDiff) error return nil } +func (b *Builder) applyModifySchemaCharsetAndCollate(m *meta.Meta, diff *model.SchemaDiff) error { + di, err := m.GetDatabase(diff.SchemaID) + if err != nil { + return errors.Trace(err) + } + if di == nil { + // This should never happen. + return ErrDatabaseNotExists.GenWithStackByArgs( + fmt.Sprintf("(Schema ID %d)", diff.SchemaID), + ) + } + newDbInfo := b.copySchemaTables(di.Name.O) + newDbInfo.Charset = di.Charset + newDbInfo.Collate = di.Collate + return nil +} + func (b *Builder) applyDropSchema(schemaID int64) []int64 { di, ok := b.is.SchemaByID(schemaID) if !ok { diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 6ea53a28e9293..21de7c0e2bef9 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -223,7 +223,7 @@ func (b *planBuilder) build(node ast.Node) (Plan, error) { *ast.GrantStmt, *ast.DropUserStmt, *ast.AlterUserStmt, *ast.RevokeStmt, *ast.KillStmt, *ast.DropStatsStmt: return b.buildSimple(node.(ast.StmtNode)), nil case ast.DDLNode: - return b.buildDDL(x), nil + return b.buildDDL(x) } return nil, ErrUnsupportedType.GenWithStack("Unsupported type %T", node) } @@ -1422,8 +1422,19 @@ func (b *planBuilder) buildLoadStats(ld *ast.LoadStatsStmt) Plan { return p } -func (b *planBuilder) buildDDL(node ast.DDLNode) Plan { +func (b *planBuilder) buildDDL(node ast.DDLNode) (Plan, error) { switch v := node.(type) { + case *ast.AlterDatabaseStmt: + if v.AlterDefaultDatabase { + v.Name = b.ctx.GetSessionVars().CurrentDB + } + if v.Name == "" { + return nil, ErrNoDB + } + b.visitInfo = append(b.visitInfo, visitInfo{ + privilege: mysql.AlterPriv, + db: v.Name, + }) case *ast.AlterTableStmt: b.visitInfo = append(b.visitInfo, visitInfo{ privilege: mysql.AlterPriv, @@ -1493,7 +1504,7 @@ func (b *planBuilder) buildDDL(node ast.DDLNode) Plan { } p := &DDL{Statement: node} - return p + return p, nil } // buildTrace builds a trace plan. Inside this method, it first optimize the diff --git a/planner/core/preprocess.go b/planner/core/preprocess.go index 350ec424b75ec..410c5c79e08b3 100644 --- a/planner/core/preprocess.go +++ b/planner/core/preprocess.go @@ -71,6 +71,8 @@ func (p *preprocessor) Enter(in ast.Node) (out ast.Node, skipChildren bool) { p.checkAlterTableGrammar(node) case *ast.CreateDatabaseStmt: p.checkCreateDatabaseGrammar(node) + case *ast.AlterDatabaseStmt: + p.checkAlterDatabaseGrammar(node) case *ast.DropDatabaseStmt: p.checkDropDatabaseGrammar(node) case *ast.ShowStmt: @@ -254,6 +256,13 @@ func (p *preprocessor) checkCreateDatabaseGrammar(stmt *ast.CreateDatabaseStmt) } } +func (p *preprocessor) checkAlterDatabaseGrammar(stmt *ast.AlterDatabaseStmt) { + // for 'ALTER DATABASE' statement, database name can be empty to alter default database. + if isIncorrectName(stmt.Name) && !stmt.AlterDefaultDatabase { + p.err = ddl.ErrWrongDBName.GenWithStackByArgs(stmt.Name) + } +} + func (p *preprocessor) checkDropDatabaseGrammar(stmt *ast.DropDatabaseStmt) { if isIncorrectName(stmt.Name) { p.err = ddl.ErrWrongDBName.GenWithStackByArgs(stmt.Name) diff --git a/planner/core/preprocess_test.go b/planner/core/preprocess_test.go index 45a8200f451d0..3c8f7c6561955 100644 --- a/planner/core/preprocess_test.go +++ b/planner/core/preprocess_test.go @@ -125,6 +125,9 @@ func (s *testValidatorSuite) TestValidator(c *C) { {"drop table `t `", true, errors.New("[ddl:1103]Incorrect table name 't '")}, {"create database ``", true, errors.New("[ddl:1102]Incorrect database name ''")}, {"create database `test `", true, errors.New("[ddl:1102]Incorrect database name 'test '")}, + {"alter database collate = 'utf8mb4_bin'", true, nil}, + {"alter database `` collate = 'utf8mb4_bin'", true, errors.New("[ddl:1102]Incorrect database name ''")}, + {"alter database `test ` collate = 'utf8mb4_bin'", true, errors.New("[ddl:1102]Incorrect database name 'test '")}, {"drop database ``", true, errors.New("[ddl:1102]Incorrect database name ''")}, {"drop database `test `", true, errors.New("[ddl:1102]Incorrect database name 'test '")}, {"alter table `t ` add column c int", true, errors.New("[ddl:1103]Incorrect table name 't '")}, diff --git a/util/admin/admin.go b/util/admin/admin.go index 16c3b7a76dc65..d569b3b49036b 100644 --- a/util/admin/admin.go +++ b/util/admin/admin.go @@ -108,7 +108,8 @@ func isJobRollbackable(job *model.Job, id int64) error { model.ActionRebaseAutoID, model.ActionShardRowID, model.ActionTruncateTable, model.ActionAddForeignKey, model.ActionDropForeignKey, model.ActionRenameTable, - model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition: + model.ActionModifyTableCharsetAndCollate, model.ActionTruncateTablePartition, + model.ActionModifySchemaCharsetAndCollate: if job.SchemaState != model.StateNone { return ErrCannotCancelDDLJob.GenWithStackByArgs(id) }