Skip to content

Commit

Permalink
*: fix utf8 charset upgrade compatibility (#9820)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored and winkyao committed Mar 25, 2019
1 parent d6490c1 commit c451f00
Show file tree
Hide file tree
Showing 17 changed files with 232 additions and 58 deletions.
34 changes: 19 additions & 15 deletions config/config.go
Original file line number Diff line number Diff line change
Expand Up @@ -74,7 +74,10 @@ type Config struct {
Binlog Binlog `toml:"binlog" json:"binlog"`
CompatibleKillQuery bool `toml:"compatible-kill-query" json:"compatible-kill-query"`
Plugin Plugin `toml:"plugin" json:"plugin"`
CheckMb4ValueInUtf8 bool `toml:"check-mb4-value-in-utf8" json:"check-mb4-value-in-utf8"`
CheckMb4ValueInUTF8 bool `toml:"check-mb4-value-in-utf8" json:"check-mb4-value-in-utf8"`
// TreatOldVersionUTF8AsUTF8MB4 is use to treat old version table/column UTF8 charset as UTF8MB4. This is for compatibility.
// Currently not support dynamic modify, because this need to reload all old version schema.
TreatOldVersionUTF8AsUTF8MB4 bool `toml:"treat-old-version-utf8-as-utf8mb4" json:"treat-old-version-utf8-as-utf8mb4"`
}

// Log is the log section of config.
Expand Down Expand Up @@ -268,20 +271,21 @@ type Plugin struct {
}

var defaultConf = Config{
Host: "0.0.0.0",
AdvertiseAddress: "",
Port: 4000,
Cors: "",
Store: "mocktikv",
Path: "/tmp/tidb",
RunDDL: true,
SplitTable: true,
Lease: "45s",
TokenLimit: 1000,
OOMAction: "log",
MemQuotaQuery: 32 << 30,
EnableStreaming: false,
CheckMb4ValueInUtf8: true,
Host: "0.0.0.0",
AdvertiseAddress: "",
Port: 4000,
Cors: "",
Store: "mocktikv",
Path: "/tmp/tidb",
RunDDL: true,
SplitTable: true,
Lease: "45s",
TokenLimit: 1000,
OOMAction: "log",
MemQuotaQuery: 32 << 30,
EnableStreaming: false,
CheckMb4ValueInUTF8: true,
TreatOldVersionUTF8AsUTF8MB4: true,
TxnLocalLatches: TxnLocalLatches{
Enabled: true,
Capacity: 2048000,
Expand Down
3 changes: 3 additions & 0 deletions config/config.toml.example
Original file line number Diff line number Diff line change
Expand Up @@ -51,6 +51,9 @@ compatible-kill-query = false
# check mb4 value in utf8 is used to control whether to check the mb4 characters when the charset is utf8.
check-mb4-value-in-utf8 = true

# treat-old-version-utf8-as-utf8mb4 use for upgrade compatibility. Set to true will treat old version table/column UTF8 charset as UTF8MB4.
treat-old-version-utf8-as-utf8mb4 = true

[log]
# Log level: debug, info, warn, error, fatal.
level = "info"
Expand Down
2 changes: 1 addition & 1 deletion config/config_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ func (s *testConfigSuite) TestConfig(c *C) {
conf.Binlog.IgnoreError = true
conf.Binlog.Strategy = "hash"
conf.TiKVClient.CommitTimeout = "10s"
conf.CheckMb4ValueInUtf8 = true
conf.CheckMb4ValueInUTF8 = true
configFile := "config.toml"
_, localFile, _, _ := runtime.Caller(0)
configFile = path.Join(path.Dir(localFile), configFile)
Expand Down
4 changes: 0 additions & 4 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -576,10 +576,6 @@ func generateOriginDefaultValue(col *model.ColumnInfo) (interface{}, error) {
if odValue == strings.ToUpper(ast.CurrentTimestamp) {
if col.Tp == mysql.TypeTimestamp {
odValue = time.Now().UTC().Format(types.TimeFormat)
// Version = 1: For OriginDefaultValue and DefaultValue of timestamp column will stores the default time in UTC time zone.
// This will fix bug in version 0.
// TODO: remove this version field after there is no old version 0.
col.Version = model.ColumnInfoVersion1
} else if col.Tp == mysql.TypeDatetime {
odValue = time.Now().Format(types.TimeFormat)
}
Expand Down
142 changes: 140 additions & 2 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,10 +24,12 @@ 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"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/ddl"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/infoschema"
Expand All @@ -40,6 +42,7 @@ import (
"github.com/pingcap/tidb/store/mockstore/mocktikv"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/mock"
"github.com/pingcap/tidb/util/testkit"
)

Expand Down Expand Up @@ -1221,8 +1224,7 @@ func (s *testIntegrationSuite) assertAlterErrorExec(c *C, sql string) {
func (s *testIntegrationSuite) TestAlterAlgorithm(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test")
s.tk.MustExec("drop table if exists t")
s.tk.MustExec("drop table if exists t1")
s.tk.MustExec("drop table if exists t, t1")
defer s.tk.MustExec("drop table if exists t")

s.tk.MustExec(`create table t(
Expand Down Expand Up @@ -1282,3 +1284,139 @@ func (s *testIntegrationSuite) TestAlterAlgorithm(c *C) {
s.assertAlterErrorExec(c, "alter table t default charset = utf8mb4, ALGORITHM=INPLACE")
s.tk.MustExec("alter table t default charset = utf8mb4, ALGORITHM=INSTANT")
}

func (s *testIntegrationSuite) TestIgnoreColumnUTF8Charset(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use test")
s.tk.MustExec("drop table if exists t")
defer s.tk.MustExec("drop table if exists t")

s.tk.MustExec("create table t (a varchar(10) character set utf8, b varchar(10) character set ascii) charset=utf8mb4;")
assertErrorCode(c, s.tk, "insert into t set a= x'f09f8c80';", mysql.ErrTruncatedWrongValueForField)
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(10) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL,\n" +
" `b` varchar(10) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

// Mock old version table info with column charset is utf8.
db, ok := domain.GetDomain(s.ctx).InfoSchema().SchemaByName(model.NewCIStr("test"))
tbl := testGetTableByName(c, s.ctx, "test", "t")
tblInfo := tbl.Meta().Clone()
tblInfo.Version = model.TableInfoVersion0
tblInfo.Columns[0].Version = model.ColumnInfoVersion0
updateTableInfo := func(tblInfo *model.TableInfo) {
mockCtx := mock.NewContext()
mockCtx.Store = s.store
err := mockCtx.NewTxn(context.Background())
c.Assert(err, IsNil)
txn, err := mockCtx.Txn(true)
c.Assert(err, IsNil)
mt := meta.NewMeta(txn)
c.Assert(ok, IsTrue)
err = mt.UpdateTable(db.ID, tblInfo)
c.Assert(err, IsNil)
err = txn.Commit(context.Background())
c.Assert(err, IsNil)
}
updateTableInfo(tblInfo)
s.tk.MustExec("alter table t add column c varchar(10) character set utf8;") // load latest schema.
c.Assert(config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4, IsTrue)
s.tk.MustExec("insert into t set a= x'f09f8c80'")
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(10) DEFAULT NULL,\n" +
" `b` varchar(10) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL,\n" +
" `c` varchar(10) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = false
s.tk.MustExec("alter table t drop column c;") // reload schema.
assertErrorCode(c, s.tk, "insert into t set a= x'f09f8c80'", mysql.ErrTruncatedWrongValueForField)
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(10) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL,\n" +
" `b` varchar(10) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

// Mock old version table info with table and column charset is utf8.
tbl = testGetTableByName(c, s.ctx, "test", "t")
tblInfo = tbl.Meta().Clone()
tblInfo.Charset = charset.CharsetUTF8
tblInfo.Collate = charset.CollationUTF8
tblInfo.Version = model.TableInfoVersion0
tblInfo.Columns[0].Version = model.ColumnInfoVersion0
updateTableInfo(tblInfo)

config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = true
s.tk.MustExec("alter table t add column c varchar(10);") // load latest schema.
s.tk.MustExec("insert into t set a= x'f09f8c80'")
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(10) DEFAULT NULL,\n" +
" `b` varchar(10) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL,\n" +
" `c` varchar(10) DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = false
s.tk.MustExec("alter table t drop column c;") // reload schema.
assertErrorCode(c, s.tk, "insert into t set a= x'f09f8c80'", mysql.ErrTruncatedWrongValueForField)
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(10) DEFAULT NULL,\n" +
" `b` varchar(10) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin"))

// Test modify column charset.
config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = true
s.tk.MustExec("alter table t modify column a varchar(10) character set utf8mb4") // change column charset.
tbl = testGetTableByName(c, s.ctx, "test", "t")
c.Assert(tbl.Meta().Columns[0].Charset, Equals, charset.CharsetUTF8MB4)
c.Assert(tbl.Meta().Columns[0].Collate, Equals, charset.CollationUTF8MB4)
c.Assert(tbl.Meta().Columns[0].Version, Equals, model.ColumnInfoVersion0)
s.tk.MustExec("insert into t set a= x'f09f8c80'")
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(10) DEFAULT NULL,\n" +
" `b` varchar(10) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
// Test for change column should not modify the column version.
s.tk.MustExec("alter table t change column a a varchar(20)") // change column.
tbl = testGetTableByName(c, s.ctx, "test", "t")
c.Assert(tbl.Meta().Columns[0].Charset, Equals, charset.CharsetUTF8MB4)
c.Assert(tbl.Meta().Columns[0].Collate, Equals, charset.CollationUTF8MB4)
c.Assert(tbl.Meta().Columns[0].Version, Equals, model.ColumnInfoVersion0)

// Test for v2.1.5 and v2.1.6 that table version is 1 but column version is 0.
tbl = testGetTableByName(c, s.ctx, "test", "t")
tblInfo = tbl.Meta().Clone()
tblInfo.Charset = charset.CharsetUTF8
tblInfo.Collate = charset.CollationUTF8
tblInfo.Version = model.TableInfoVersion1
tblInfo.Columns[0].Version = model.ColumnInfoVersion0
tblInfo.Columns[0].Charset = charset.CharsetUTF8
tblInfo.Columns[0].Collate = charset.CollationUTF8
updateTableInfo(tblInfo)
c.Assert(config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4, IsTrue)
s.tk.MustExec("alter table t change column b b varchar(20) character set ascii") // reload schema.
s.tk.MustExec("insert into t set a= x'f09f8c80'")
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(20) DEFAULT NULL,\n" +
" `b` varchar(20) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))

config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = false
s.tk.MustExec("alter table t change column b b varchar(30) character set ascii") // reload schema.
assertErrorCode(c, s.tk, "insert into t set a= x'f09f8c80'", mysql.ErrTruncatedWrongValueForField)
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(20) DEFAULT NULL,\n" +
" `b` varchar(30) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8 COLLATE=utf8_bin"))

// Test for alter table convert charset
config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = true
s.tk.MustExec("alter table t change column b b varchar(40) character set ascii") // reload schema.
s.tk.MustExec("alter table t convert to charset utf8mb4;")

config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 = false
s.tk.MustExec("alter table t change column b b varchar(50) character set ascii") // reload schema.
// TODO: fix this after PR 9790.
s.tk.MustQuery("show create table t").Check(testkit.Rows("t CREATE TABLE `t` (\n" +
" `a` varchar(20) CHARACTER SET utf8 COLLATE utf8_bin DEFAULT NULL,\n" +
" `b` varchar(50) CHARACTER SET ascii COLLATE ascii_bin DEFAULT NULL\n" +
") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin"))
}
30 changes: 16 additions & 14 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -337,10 +337,6 @@ func convertTimestampDefaultValToUTC(ctx sessionctx.Context, defaultVal interfac
return defaultVal, errors.Trace(err)
}
defaultVal = t.String()
// Version = 1: For OriginDefaultValue and DefaultValue of timestamp column will stores the default time in UTC time zone.
// This will fix bug in version 0.
// TODO: remove this version field after there is no old version 0.
col.Version = model.ColumnInfoVersion1
}
}
return defaultVal, nil
Expand All @@ -362,6 +358,8 @@ func columnDefToCol(ctx sessionctx.Context, offset int, colDef *ast.ColumnDef, o
Offset: offset,
Name: colDef.Name.Name,
FieldType: *colDef.Tp,
// TODO: remove this version field after there is no old version.
Version: model.CurrLatestColumnInfoVersion,
})

if !isExplicitTimeStamp() {
Expand Down Expand Up @@ -1302,19 +1300,21 @@ func buildViewInfoWithTableColumns(ctx sessionctx.Context, s *ast.CreateViewStmt
if s.Cols == nil {
for i, v := range schemaCols {
tableColumns[i] = table.ToColumn(&model.ColumnInfo{
Name: v.AsName,
ID: int64(i),
Offset: i,
State: model.StatePublic,
Name: v.AsName,
ID: int64(i),
Offset: i,
State: model.StatePublic,
Version: model.CurrLatestColumnInfoVersion,
})
}
} else {
for i, v := range s.Cols {
tableColumns[i] = table.ToColumn(&model.ColumnInfo{
Name: v,
ID: int64(i),
Offset: i,
State: model.StatePublic,
Name: v,
ID: int64(i),
Offset: i,
State: model.StatePublic,
Version: model.CurrLatestColumnInfoVersion,
})
}
}
Expand Down Expand Up @@ -2316,6 +2316,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or
OriginDefaultValue: col.OriginDefaultValue,
FieldType: *specNewColumn.Tp,
Name: newColName,
Version: col.Version,
})

// TODO: Remove it when all table versions are greater than or equal to TableInfoVersion1.
Expand Down Expand Up @@ -2551,8 +2552,9 @@ func (d *ddl) AlterTableCharsetAndCollate(ctx sessionctx.Context, ident ast.Iden
return errors.Trace(err)
}
}

if origCharset == toCharset && origCollate == toCollate {
// Old version schema charset maybe modified when load schema if TreatOldVersionUTF8AsUTF8MB4 was enable.
// So even if the origCharset equal toCharset, we still need to do the ddl for old version schema.
if origCharset == toCharset && origCollate == toCollate && tb.Meta().Version >= model.TableInfoVersion2 {
// nothing to do.
return nil
}
Expand Down
6 changes: 6 additions & 0 deletions domain/domain.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,12 @@ func (do *Domain) fetchSchemasWithTables(schemas []*model.DBInfo, m *meta.Meta,
done <- err
return
}
// If TreatOldVersionUTF8AsUTF8MB4 was enable, need to convert the old version schema UTF8 charset to UTF8MB4.
if config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 {
for _, tbInfo := range tables {
infoschema.ConvertOldVersionUTF8ToUTF8MB4IfNeed(tbInfo)
}
}
di.Tables = make([]*model.TableInfo, 0, len(tables))
for _, tbl := range tables {
if tbl.State != model.StatePublic {
Expand Down
4 changes: 2 additions & 2 deletions executor/statement_context_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -102,9 +102,9 @@ func (s *testSuite1) TestStatementContext(c *C) {
_, err = tk.Exec("insert t1 values (unhex('F0A48BAE'))")
c.Assert(err, NotNil)
c.Assert(terror.ErrorEqual(err, table.ErrTruncateWrongValue), IsTrue, Commentf("err %v", err))
config.GetGlobalConfig().CheckMb4ValueInUtf8 = false
config.GetGlobalConfig().CheckMb4ValueInUTF8 = false
tk.MustExec("insert t1 values (unhex('f09f8c80'))")
config.GetGlobalConfig().CheckMb4ValueInUtf8 = true
config.GetGlobalConfig().CheckMb4ValueInUTF8 = true
_, err = tk.Exec("insert t1 values (unhex('F0A48BAE'))")
c.Assert(err, NotNil)
}
21 changes: 21 additions & 0 deletions infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,9 @@ import (
"sort"

"github.com/pingcap/errors"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/table"
Expand Down Expand Up @@ -170,6 +172,8 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i
fmt.Sprintf("(Table ID %d)", tableID),
)
}
ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo)

if alloc == nil {
schemaID := dbInfo.ID
alloc = autoid.NewAllocator(b.handle.store, tblInfo.GetDBID(schemaID), tblInfo.IsAutoIncColUnsigned())
Expand All @@ -193,6 +197,23 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i
return nil
}

// ConvertOldVersionUTF8ToUTF8MB4IfNeed convert old version UTF8 to UTF8MB4 if config.TreatOldVersionUTF8AsUTF8MB4 is enable.
func ConvertOldVersionUTF8ToUTF8MB4IfNeed(tbInfo *model.TableInfo) {
if !config.GetGlobalConfig().TreatOldVersionUTF8AsUTF8MB4 || tbInfo.Version >= model.TableInfoVersion2 {
return
}
if tbInfo.Charset == charset.CharsetUTF8 {
tbInfo.Charset = charset.CharsetUTF8MB4
tbInfo.Collate = charset.CollationUTF8MB4
}
for _, col := range tbInfo.Columns {
if col.Version < model.ColumnInfoVersion2 && col.Charset == charset.CharsetUTF8 {
col.Charset = charset.CharsetUTF8MB4
col.Collate = charset.CollationUTF8MB4
}
}
}

func (b *Builder) applyDropTable(dbInfo *model.DBInfo, tableID int64) {
bucketIdx := tableBucketIdx(tableID)
sortedTbls := b.is.sortedTablesBuckets[bucketIdx]
Expand Down
4 changes: 2 additions & 2 deletions server/http_handler.go
Original file line number Diff line number Diff line change
Expand Up @@ -761,9 +761,9 @@ func (h settingsHandler) ServeHTTP(w http.ResponseWriter, req *http.Request) {
if checkMb4ValueInUtf8 := req.Form.Get("check_mb4_value_in_utf8"); checkMb4ValueInUtf8 != "" {
switch checkMb4ValueInUtf8 {
case "0":
config.GetGlobalConfig().CheckMb4ValueInUtf8 = false
config.GetGlobalConfig().CheckMb4ValueInUTF8 = false
case "1":
config.GetGlobalConfig().CheckMb4ValueInUtf8 = true
config.GetGlobalConfig().CheckMb4ValueInUTF8 = true
default:
writeError(w, errors.New("illegal argument"))
return
Expand Down
Loading

0 comments on commit c451f00

Please sign in to comment.