Skip to content

Commit

Permalink
ddl: implement ALTER DATABASE to alter charset/collation (pingcap#1…
Browse files Browse the repository at this point in the history
  • Loading branch information
bb7133 committed May 16, 2019
1 parent 205418a commit 2a955ab
Show file tree
Hide file tree
Showing 14 changed files with 269 additions and 13 deletions.
106 changes: 106 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package ddl_test
import (
"context"
"fmt"
"strings"

. "github.com/pingcap/check"
"github.com/pingcap/errors"
Expand Down Expand Up @@ -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")
}
4 changes: 4 additions & 0 deletions ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -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])

Expand Down Expand Up @@ -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
Expand Down Expand Up @@ -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)
Expand Down
57 changes: 57 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
2 changes: 2 additions & 0 deletions ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -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:
Expand Down
3 changes: 2 additions & 1 deletion ddl/rollingback.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand Down
31 changes: 31 additions & 0 deletions ddl/schema.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 {
Expand Down
24 changes: 16 additions & 8 deletions executor/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -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))
Expand Down Expand Up @@ -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)
Expand Down
2 changes: 2 additions & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -78,3 +78,5 @@ require (
gopkg.in/natefinch/lumberjack.v2 v2.0.0
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect
)

replace github.com/pingcap/parser => github.com/bb7133/parser v0.0.0-20190515111246-584f56a83b95
2 changes: 2 additions & 0 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -3,6 +3,8 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03
github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
github.com/apache/thrift v0.0.0-20161221203622-b2a4d4ae21c7 h1:Fv9bK1Q+ly/ROk4aJsVMeuIwPel4bEnD8EPiI91nZMg=
github.com/apache/thrift v0.0.0-20161221203622-b2a4d4ae21c7/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ=
github.com/bb7133/parser v0.0.0-20190515111246-584f56a83b95 h1:7Bdapqatt2/DhqC/arvWd7JRth+0pjA9mVj5hmN+Vog=
github.com/bb7133/parser v0.0.0-20190515111246-584f56a83b95/go.mod h1:Vavbz+Ti/y+IQGXwsgGMHASfCyvH4A6zVQspCFBuRXM=
github.com/beorn7/perks v0.0.0-20160229213445-3ac7bf7a47d1 h1:OnJHjoVbY69GG4gclp0ngXfywigLhR6rrgUxmxQRWO4=
github.com/beorn7/perks v0.0.0-20160229213445-3ac7bf7a47d1/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q=
github.com/blacktear23/go-proxyprotocol v0.0.0-20180807104634-af7a81e8dd0d h1:rQlvB2AYWme2bIB18r/SipGiMEVJYE9U0z+MGoU/LtQ=
Expand Down
19 changes: 19 additions & 0 deletions infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down Expand Up @@ -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 {
Expand Down
17 changes: 14 additions & 3 deletions planner/core/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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)
}
Expand Down Expand Up @@ -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,
Expand Down Expand Up @@ -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
Expand Down
Loading

0 comments on commit 2a955ab

Please sign in to comment.