diff --git a/config/config.go b/config/config.go index 6ab636cff3d01..8edaea6be2fd3 100644 --- a/config/config.go +++ b/config/config.go @@ -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. @@ -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, diff --git a/config/config.toml.example b/config/config.toml.example index 6c5e8bda1d8ce..d6195caec15f7 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -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" diff --git a/config/config_test.go b/config/config_test.go index aa3a91496b09c..d9e1ec2aba6de 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -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) diff --git a/ddl/column.go b/ddl/column.go index 917633b2c1232..d20d5390e8a2d 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -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) } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index 6f69f2d9bc820..2f2f0f2b000c0 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -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" @@ -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" ) @@ -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( @@ -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")) +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 1ab6a84ee49d1..c80bf5608a554 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -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 @@ -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() { @@ -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, }) } } @@ -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. @@ -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 } diff --git a/domain/domain.go b/domain/domain.go index f1d0207c26cef..9813ebba40b0e 100644 --- a/domain/domain.go +++ b/domain/domain.go @@ -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 { diff --git a/executor/statement_context_test.go b/executor/statement_context_test.go index 121d25d311e1b..a3593105d8fe1 100644 --- a/executor/statement_context_test.go +++ b/executor/statement_context_test.go @@ -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) } diff --git a/infoschema/builder.go b/infoschema/builder.go index cbef60fe64a7d..17b977bbecb66 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -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" @@ -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()) @@ -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] diff --git a/server/http_handler.go b/server/http_handler.go index 6349b1ab5fce0..b12c2806aefda 100644 --- a/server/http_handler.go +++ b/server/http_handler.go @@ -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 diff --git a/server/http_handler_test.go b/server/http_handler_test.go index 151bbdd802109..d3106df0ed69b 100644 --- a/server/http_handler_test.go +++ b/server/http_handler_test.go @@ -660,20 +660,20 @@ func (ts *HTTPHandlerTestSuite) TestPostSettings(c *C) { resp, err = http.PostForm("http://127.0.0.1:10090/settings", form) c.Assert(err, IsNil) c.Assert(resp.StatusCode, Equals, http.StatusOK) - c.Assert(config.GetGlobalConfig().CheckMb4ValueInUtf8, Equals, true) + c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, true) txn1, err := dbt.db.Begin() c.Assert(err, IsNil) _, err = txn1.Exec("insert t2 values (unhex('F0A48BAE'));") c.Assert(err, NotNil) txn1.Commit() - // Disable CheckMb4ValueInUtf8. + // Disable CheckMb4ValueInUTF8. form = make(url.Values) form.Set("check_mb4_value_in_utf8", "0") resp, err = http.PostForm("http://127.0.0.1:10090/settings", form) c.Assert(err, IsNil) c.Assert(resp.StatusCode, Equals, http.StatusOK) - c.Assert(config.GetGlobalConfig().CheckMb4ValueInUtf8, Equals, false) + c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, false) dbt.mustExec("insert t2 values (unhex('f09f8c80'));") } diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 36f0cec8d196f..936c1d4326ffd 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -721,8 +721,8 @@ func (s *SessionVars) SetSystemVar(name string, val string) error { s.EnableRadixJoin = TiDBOptOn(val) case TiDBEnableWindowFunction: s.EnableWindowFunction = TiDBOptOn(val) - case TiDBCheckMb4ValueInUtf8: - config.GetGlobalConfig().CheckMb4ValueInUtf8 = TiDBOptOn(val) + case TiDBCheckMb4ValueInUTF8: + config.GetGlobalConfig().CheckMb4ValueInUTF8 = TiDBOptOn(val) case TiDBSlowQueryFile: s.SlowQueryFile = val } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index d0ffdb6a31ee3..2fcc5d6331c15 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -684,7 +684,7 @@ var defaultSysVars = []*SysVar{ {ScopeSession, TiDBDDLReorgPriority, "PRIORITY_LOW"}, {ScopeSession, TiDBForcePriority, mysql.Priority2Str[DefTiDBForcePriority]}, {ScopeSession, TiDBEnableRadixJoin, BoolToIntStr(DefTiDBUseRadixJoin)}, - {ScopeSession, TiDBCheckMb4ValueInUtf8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUtf8)}, + {ScopeSession, TiDBCheckMb4ValueInUTF8, BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8)}, {ScopeSession, TiDBSlowQueryFile, ""}, } diff --git a/sessionctx/variable/tidb_vars.go b/sessionctx/variable/tidb_vars.go index 6ad308e435e7f..9ca6d7b338e4f 100644 --- a/sessionctx/variable/tidb_vars.go +++ b/sessionctx/variable/tidb_vars.go @@ -129,8 +129,8 @@ const ( // off: always disable table partition. TiDBEnableTablePartition = "tidb_enable_table_partition" - // TiDBCheckMb4ValueInUtf8 is used to control whether to enable the check wrong utf8 value. - TiDBCheckMb4ValueInUtf8 = "tidb_check_mb4_value_in_utf8" + // TiDBCheckMb4ValueInUTF8 is used to control whether to enable the check wrong utf8 value. + TiDBCheckMb4ValueInUTF8 = "tidb_check_mb4_value_in_utf8" ) // TiDB system variable names that both in session and global scope. diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index db8cce5ba7ed9..425bb9381c576 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -122,8 +122,8 @@ func GetSessionOnlySysVars(s *SessionVars, key string) (string, bool, error) { return config.GetGlobalConfig().Plugin.Dir, true, nil case PluginLoad: return config.GetGlobalConfig().Plugin.Load, true, nil - case TiDBCheckMb4ValueInUtf8: - return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUtf8), true, nil + case TiDBCheckMb4ValueInUTF8: + return BoolToIntStr(config.GetGlobalConfig().CheckMb4ValueInUTF8), true, nil } sVal, ok := s.systems[key] if ok { @@ -361,7 +361,8 @@ func ValidateSetSystemVar(vars *SessionVars, name string, value string) (string, case AutocommitVar, TiDBSkipUTF8Check, TiDBOptAggPushDown, TiDBOptInSubqToJoinAndAgg, TiDBBatchInsert, TiDBDisableTxnAutoRetry, TiDBEnableStreaming, - TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, TiDBCheckMb4ValueInUtf8: + TiDBBatchDelete, TiDBBatchCommit, TiDBEnableCascadesPlanner, TiDBEnableWindowFunction, + TiDBCheckMb4ValueInUTF8: if strings.EqualFold(value, "ON") || value == "1" || strings.EqualFold(value, "OFF") || value == "0" { return value, nil } diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index e94fdffb08463..7dd8f5249b278 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -243,14 +243,16 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { c.Assert(val, Equals, "on") c.Assert(v.EnableTablePartition, Equals, "on") - SetSessionSystemVar(v, TiDBCheckMb4ValueInUtf8, types.NewStringDatum("1")) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUtf8) + err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, types.NewStringDatum("1")) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, "1") - c.Assert(config.GetGlobalConfig().CheckMb4ValueInUtf8, Equals, true) - SetSessionSystemVar(v, TiDBCheckMb4ValueInUtf8, types.NewStringDatum("0")) - val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUtf8) + c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, true) + err = SetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8, types.NewStringDatum("0")) + c.Assert(err, IsNil) + val, err = GetSessionSystemVar(v, TiDBCheckMb4ValueInUTF8) c.Assert(err, IsNil) c.Assert(val, Equals, "0") - c.Assert(config.GetGlobalConfig().CheckMb4ValueInUtf8, Equals, false) + c.Assert(config.GetGlobalConfig().CheckMb4ValueInUTF8, Equals, false) } diff --git a/table/column.go b/table/column.go index 4c614d6a7a753..49dd68e7fd8c4 100644 --- a/table/column.go +++ b/table/column.go @@ -183,6 +183,7 @@ func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo) ( } str := casted.GetString() utf8Charset := col.Charset == mysql.UTF8Charset + doMB4CharCheck := utf8Charset && config.GetGlobalConfig().CheckMb4ValueInUTF8 for i, w := 0, 0; i < len(str); i += w { runeValue, width := utf8.DecodeRuneInString(str[i:]) if runeValue == utf8.RuneError { @@ -192,7 +193,7 @@ func CastValue(ctx sessionctx.Context, val types.Datum, col *model.ColumnInfo) ( } casted, err = handleWrongUtf8Value(ctx, col, &casted, str, i) break - } else if width > 3 && utf8Charset && config.GetGlobalConfig().CheckMb4ValueInUtf8 { + } else if width > 3 && doMB4CharCheck { // Handle non-BMP characters. casted, err = handleWrongUtf8Value(ctx, col, &casted, str, i) break