Skip to content

Commit

Permalink
ddl: fix a bug in AlterSchema job and add more tests (pingcap#10529)
Browse files Browse the repository at this point in the history
  • Loading branch information
bb7133 authored May 20, 2019
1 parent 2d70e4d commit 43a5cf2
Show file tree
Hide file tree
Showing 5 changed files with 57 additions and 6 deletions.
18 changes: 16 additions & 2 deletions ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -63,7 +63,7 @@ func (s *testStateChangeSuite) SetUpSuite(c *C) {
c.Assert(err, IsNil)
s.se, err = session.CreateSession4Test(s.store)
c.Assert(err, IsNil)
_, err = s.se.Execute(context.Background(), "create database test_db_state")
_, err = s.se.Execute(context.Background(), "create database test_db_state default charset utf8 default collate utf8_bin")
c.Assert(err, IsNil)
_, err = s.se.Execute(context.Background(), "use test_db_state")
c.Assert(err, IsNil)
Expand Down Expand Up @@ -442,7 +442,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:210]unsupported modify charset from utf8mb4 to binary")
c.Assert(err.Error(), Equals, "[ddl:210]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)
Expand Down Expand Up @@ -975,3 +975,17 @@ func (s *testStateChangeSuite) TestParallelDDLBeforeRunDDLJob(c *C) {
intercept = &ddl.TestInterceptor{}
d.(ddl.DDLForTest).SetInterceptoror(intercept)
}

func (s *testStateChangeSuite) TestParallelAlterSchemaCharsetAndCollate(c *C) {
sql := "ALTER SCHEMA test_db_state CHARSET utf8mb4 COLLATE utf8mb4_general_ci"
f := func(c *C, err1, err2 error) {
c.Assert(err1, IsNil)
c.Assert(err2, IsNil)
}
s.testControlParallelExecSQL(c, sql, sql, f)
sql = `SELECT default_character_set_name, default_collation_name
FROM information_schema.schemata
WHERE schema_name='test_db_state'`
tk := testkit.NewTestKit(c, s.store)
tk.MustQuery(sql).Check(testkit.Rows("utf8mb4 utf8mb4_general_ci"))
}
2 changes: 1 addition & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -127,7 +127,7 @@ func (d *ddl) AlterSchema(ctx sessionctx.Context, stmt *ast.AlterDatabaseStmt) (
if !ok {
return infoschema.ErrDatabaseNotExists.GenWithStackByArgs(dbName.O)
}
if dbInfo.Charset == toCharset && dbInfo.Charset == toCollate {
if dbInfo.Charset == toCharset && dbInfo.Collate == toCollate {
return nil
}

Expand Down
23 changes: 21 additions & 2 deletions ddl/ddl_worker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -395,9 +395,10 @@ func buildCancelJobTests(firstID int64) []testCancelJob {

{act: model.ActionModifyTableCharsetAndCollate, jobIDs: []int64{firstID + 25}, cancelRetErrs: noErrs, cancelState: model.StateNone},
{act: model.ActionModifyTableCharsetAndCollate, jobIDs: []int64{firstID + 26}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 26)}, cancelState: model.StatePublic},

{act: model.ActionTruncateTablePartition, jobIDs: []int64{firstID + 27}, cancelRetErrs: noErrs, cancelState: model.StateNone},
{act: model.ActionTruncateTablePartition, jobIDs: []int64{firstID + 28}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 28)}, cancelState: model.StatePublic},
{act: model.ActionModifySchemaCharsetAndCollate, jobIDs: []int64{firstID + 30}, cancelRetErrs: noErrs, cancelState: model.StateNone},
{act: model.ActionModifySchemaCharsetAndCollate, jobIDs: []int64{firstID + 31}, cancelRetErrs: []error{admin.ErrCancelFinishedDDLJob.GenWithStackByArgs(firstID + 31)}, cancelState: model.StatePublic},
}

return tests
Expand Down Expand Up @@ -726,6 +727,25 @@ func (s *testDDLSuite) TestCancelJob(c *C) {
c.Check(checkErr, IsNil)
changedTable = testGetTable(c, d, dbInfo.ID, partitionTblInfo.ID)
c.Assert(changedTable.Meta().Partition.Definitions[0].ID == partitionTblInfo.Partition.Definitions[0].ID, IsFalse)

// test modify schema charset failed caused by canceled.
test = &tests[26]
charsetAndCollate := []interface{}{"utf8mb4", "utf8mb4_bin"}
doDDLJobErrWithSchemaState(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, charsetAndCollate, &test.cancelState)
c.Check(checkErr, IsNil)
dbInfo, err = testGetSchemaInfoWithError(d, dbInfo.ID)
c.Assert(err, IsNil)
c.Assert(dbInfo.Charset, Equals, "")
c.Assert(dbInfo.Collate, Equals, "")

// test modify table charset successfully.
test = &tests[27]
doDDLJobSuccess(ctx, d, c, dbInfo.ID, tblInfo.ID, test.act, charsetAndCollate)
c.Check(checkErr, IsNil)
dbInfo, err = testGetSchemaInfoWithError(d, dbInfo.ID)
c.Assert(err, IsNil)
c.Assert(dbInfo.Charset, Equals, "utf8mb4")
c.Assert(dbInfo.Collate, Equals, "utf8mb4_bin")
}

func (s *testDDLSuite) TestIgnorableSpec(c *C) {
Expand Down Expand Up @@ -835,7 +855,6 @@ func (s *testDDLSuite) TestParallelDDL(c *C) {
ctx := testNewContext(d)
err := ctx.NewTxn(context.Background())
c.Assert(err, IsNil)

/*
build structure:
DBs -> {
Expand Down
2 changes: 1 addition & 1 deletion ddl/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -123,7 +123,7 @@ func onModifySchemaCharsetAndCollate(t *meta.Meta, job *model.Job) (ver int64, _
}

if dbInfo.Charset == toCharset && dbInfo.Collate == toCollate {
job.State = model.JobStateCancelled
job.FinishDBJob(model.JobStateDone, model.StatePublic, ver, dbInfo)
return ver, nil
}

Expand Down
18 changes: 18 additions & 0 deletions ddl/schema_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"time"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/infoschema"
Expand Down Expand Up @@ -262,3 +263,20 @@ func (s *testSchemaSuite) TestSchemaResume(c *C) {
testRunInterruptedJob(c, d1, job)
testCheckSchemaState(c, d1, dbInfo, model.StateNone)
}

func testGetSchemaInfoWithError(d *ddl, schemaID int64) (*model.DBInfo, error) {
var dbInfo *model.DBInfo
err := kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error {
t := meta.NewMeta(txn)
var err1 error
dbInfo, err1 = t.GetDatabase(schemaID)
if err1 != nil {
return errors.Trace(err1)
}
return nil
})
if err != nil {
return nil, errors.Trace(err)
}
return dbInfo, nil
}

0 comments on commit 43a5cf2

Please sign in to comment.