From b10faca6ff89a06b559e659581abd8758a6f7ed2 Mon Sep 17 00:00:00 2001 From: tangenta Date: Thu, 16 Jul 2020 10:32:58 +0800 Subject: [PATCH] config: remove allow-auto-random config option (#16596) (#18613) --- config/config.go | 6 --- config/config.toml.example | 2 - config/config_test.go | 15 ------ ddl/column.go | 46 +++++++++++++++-- ddl/column_change_test.go | 72 +++++++++++++++++++++++++++ ddl/ddl_api.go | 50 +++++++++++-------- ddl/serial_test.go | 61 +++++++++++++++++++---- executor/insert_test.go | 38 ++++---------- executor/seqtest/seq_executor_test.go | 7 +-- go.mod | 9 ++-- go.sum | 17 ++++++- meta/autoid/errors.go | 8 +-- meta/meta.go | 2 +- util/testutil/testutil.go | 5 +- 14 files changed, 230 insertions(+), 108 deletions(-) diff --git a/config/config.go b/config/config.go index 13d7ba3d74e1b..9deb76535c0d0 100644 --- a/config/config.go +++ b/config/config.go @@ -541,8 +541,6 @@ type IsolationRead struct { // Experimental controls the features that are still experimental: their semantics, interfaces are subject to change. // Using these features in the production environment is not recommended. type Experimental struct { - // Whether enable the syntax like `auto_random(3)` on the primary key column. - AllowAutoRandom bool `toml:"allow-auto-random" json:"allow-auto-random"` // Whether enable creating expression index. AllowsExpressionIndex bool `toml:"allow-expression-index" json:"allow-expression-index"` } @@ -681,7 +679,6 @@ var defaultConf = Config{ Engines: []string{"tikv", "tiflash", "tidb"}, }, Experimental: Experimental{ - AllowAutoRandom: false, AllowsExpressionIndex: false, }, EnableCollectExecutionInfo: true, @@ -867,9 +864,6 @@ func (c *Config) Valid() error { return fmt.Errorf("refresh-interval in [stmt-summary] should be greater than 0") } - if c.AlterPrimaryKey && c.Experimental.AllowAutoRandom { - return fmt.Errorf("allow-auto-random is unavailable when alter-primary-key is enabled") - } if c.PreparedPlanCache.Capacity < 1 { return fmt.Errorf("capacity in [prepared-plan-cache] should be at least 1") } diff --git a/config/config.toml.example b/config/config.toml.example index e531a4a5627e9..a05e6c06bfdf2 100644 --- a/config/config.toml.example +++ b/config/config.toml.example @@ -422,8 +422,6 @@ history-size = 24 # experimental section controls the features that are still experimental: their semantics, # interfaces are subject to change, using these features in the production environment is not recommended. [experimental] -# enable column attribute `auto_random` to be defined on the primary key column. -allow-auto-random = false # enable creating expression index. allow-expression-index = false diff --git a/config/config_test.go b/config/config_test.go index c5b544884b662..b61ec330e8421 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -205,7 +205,6 @@ max-sql-length=1024 refresh-interval=100 history-size=100 [experimental] -allow-auto-random = true allow-expression-index = true [isolation-read] engines = ["tiflash"] @@ -245,7 +244,6 @@ engines = ["tiflash"] c.Assert(conf.MaxServerConnections, Equals, uint32(200)) c.Assert(conf.MemQuotaQuery, Equals, int64(10000)) c.Assert(conf.Experimental.AllowsExpressionIndex, IsTrue) - c.Assert(conf.Experimental.AllowAutoRandom, IsTrue) c.Assert(conf.IsolationRead.Engines, DeepEquals, []string{"tiflash"}) c.Assert(conf.MaxIndexLength, Equals, 3080) c.Assert(conf.SkipRegisterToDashboard, Equals, true) @@ -414,19 +412,6 @@ func (s *testConfigSuite) TestTxnTotalSizeLimitValid(c *C) { } } -func (s *testConfigSuite) TestAllowAutoRandomValid(c *C) { - conf := NewConfig() - checkValid := func(allowAlterPK, allowAutoRand, shouldBeValid bool) { - conf.AlterPrimaryKey = allowAlterPK - conf.Experimental.AllowAutoRandom = allowAutoRand - c.Assert(conf.Valid() == nil, Equals, shouldBeValid) - } - checkValid(true, true, false) - checkValid(true, false, true) - checkValid(false, true, true) - checkValid(false, false, true) -} - func (s *testConfigSuite) TestPreparePlanCacheValid(c *C) { conf := NewConfig() tests := map[PreparedPlanCache]bool{ diff --git a/ddl/column.go b/ddl/column.go index 3137af944e93f..3a701bc56d654 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -15,10 +15,12 @@ package ddl import ( "fmt" + "math/bits" "strings" "sync/atomic" "time" + "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" @@ -27,6 +29,7 @@ import ( "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/meta" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" @@ -334,17 +337,20 @@ func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ erro oldColName := &model.CIStr{} pos := &ast.ColumnPosition{} var modifyColumnTp byte - err := job.DecodeArgs(newCol, oldColName, pos, &modifyColumnTp) + var updatedAutoRandomBits uint64 + err := job.DecodeArgs(newCol, oldColName, pos, &modifyColumnTp, &updatedAutoRandomBits) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) } - return w.doModifyColumn(t, job, newCol, oldColName, pos, modifyColumnTp) + return w.doModifyColumn(t, job, newCol, oldColName, pos, modifyColumnTp, updatedAutoRandomBits) } // doModifyColumn updates the column information and reorders all columns. -func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, pos *ast.ColumnPosition, modifyColumnTp byte) (ver int64, _ error) { +func (w *worker) doModifyColumn( + t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr, + pos *ast.ColumnPosition, modifyColumnTp byte, newAutoRandBits uint64) (ver int64, _ error) { dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job) if err != nil { return ver, errors.Trace(err) @@ -386,6 +392,12 @@ func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.Colu } }) + if newAutoRandBits > 0 { + if err := checkAndApplyNewAutoRandomBits(job, t, tblInfo, newCol, oldName, newAutoRandBits); err != nil { + return ver, errors.Trace(err) + } + } + // Column from null to not null. if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) { noPreventNullFlag := !mysql.HasPreventNullInsertFlag(oldCol.Flag) @@ -476,6 +488,34 @@ func (w *worker) doModifyColumn(t *meta.Meta, job *model.Job, newCol *model.Colu return ver, nil } +func checkAndApplyNewAutoRandomBits(job *model.Job, t *meta.Meta, tblInfo *model.TableInfo, + newCol *model.ColumnInfo, oldName *model.CIStr, newAutoRandBits uint64) error { + schemaID := job.SchemaID + newLayout := autoid.NewAutoRandomIDLayout(&newCol.FieldType, newAutoRandBits) + + // GenAutoRandomID first to prevent concurrent update. + _, err := t.GenAutoRandomID(schemaID, tblInfo.ID, 1) + if err != nil { + return err + } + currentIncBitsVal, err := t.GetAutoRandomID(schemaID, tblInfo.ID) + if err != nil { + return err + } + // Find the max number of available shard bits by + // counting leading zeros in current inc part of auto_random ID. + availableBits := bits.LeadingZeros64(uint64(currentIncBitsVal)) + isOccupyingIncBits := newLayout.TypeBitsLength-newLayout.IncrementalBits > uint64(availableBits) + if isOccupyingIncBits { + availableBits := mathutil.Min(autoid.MaxAutoRandomBits, availableBits) + errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, availableBits, newAutoRandBits, oldName.O) + job.State = model.JobStateCancelled + return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } + tblInfo.AutoRandomBits = newAutoRandBits + return nil +} + // checkForNullValue ensure there are no null values of the column of this table. // `isDataTruncated` indicates whether the new field and the old field type are the same, in order to be compatible with mysql. func checkForNullValue(ctx sessionctx.Context, isDataTruncated bool, schema, table, newCol model.CIStr, oldCols ...*model.ColumnInfo) error { diff --git a/ddl/column_change_test.go b/ddl/column_change_test.go index e72ae945a3196..8865adede8c08 100644 --- a/ddl/column_change_test.go +++ b/ddl/column_change_test.go @@ -17,12 +17,14 @@ import ( "context" "fmt" "sync" + "sync/atomic" "time" . "github.com/pingcap/check" "github.com/pingcap/errors" "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" + "github.com/pingcap/parser/mysql" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" @@ -151,6 +153,76 @@ func (s *testColumnChangeSuite) TestColumnChange(c *C) { s.testAddColumnNoDefault(c, ctx, d, tblInfo) } +func (s *testColumnChangeSuite) TestModifyAutoRandColumnWithMetaKeyChanged(c *C) { + d := testNewDDLAndStart( + context.Background(), + c, + WithStore(s.store), + WithLease(testLease), + ) + defer d.Stop() + + ids, err := d.genGlobalIDs(1) + tableID := ids[0] + c.Assert(err, IsNil) + colInfo := &model.ColumnInfo{ + Name: model.NewCIStr("a"), + Offset: 0, + State: model.StatePublic, + FieldType: *types.NewFieldType(mysql.TypeLonglong), + } + tblInfo := &model.TableInfo{ + ID: tableID, + Name: model.NewCIStr("auto_random_table_name"), + Columns: []*model.ColumnInfo{colInfo}, + AutoRandomBits: 5, + } + colInfo.ID = allocateColumnID(tblInfo) + ctx := testNewContext(d) + testCreateTable(c, ctx, d, s.dbInfo, tblInfo) + + tc := &TestDDLCallback{} + var errCount int32 = 3 + var genAutoRandErr error + tc.onJobRunBefore = func(job *model.Job) { + if atomic.LoadInt32(&errCount) > 0 && job.Type == model.ActionModifyColumn { + atomic.AddInt32(&errCount, -1) + genAutoRandErr = kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + _, err1 := t.GenAutoRandomID(s.dbInfo.ID, tableID, 1) + return err1 + }) + } + } + d.SetHook(tc) + const newAutoRandomBits uint64 = 10 + job := &model.Job{ + SchemaID: s.dbInfo.ID, + TableID: tblInfo.ID, + SchemaName: s.dbInfo.Name.L, + Type: model.ActionModifyColumn, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{colInfo, colInfo.Name, ast.ColumnPosition{}, 0, newAutoRandomBits}, + } + err = d.doDDLJob(ctx, job) + c.Assert(err, IsNil) + c.Assert(errCount == 0, IsTrue) + c.Assert(genAutoRandErr, IsNil) + testCheckJobDone(c, d, job, true) + var newTbInfo *model.TableInfo + err = kv.RunInNewTxn(d.store, false, func(txn kv.Transaction) error { + t := meta.NewMeta(txn) + var err error + newTbInfo, err = t.GetTable(s.dbInfo.ID, tableID) + if err != nil { + return errors.Trace(err) + } + return nil + }) + c.Assert(err, IsNil) + c.Assert(newTbInfo.AutoRandomBits, Equals, newAutoRandomBits) +} + func (s *testColumnChangeSuite) testAddColumnNoDefault(c *C, ctx sessionctx.Context, d *ddl, tblInfo *model.TableInfo) { tc := &TestDDLCallback{} // set up hook diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 69cc92d079086..0d290cb22f2aa 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1106,13 +1106,9 @@ func checkConstraintNames(constraints []*ast.Constraint) error { } func setTableAutoRandomBits(ctx sessionctx.Context, tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) error { - allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom pkColName := tbInfo.GetPkName() for _, col := range colDefs { if containsColumnOption(col, ast.ColumnOptionAutoRandom) { - if !allowAutoRandom { - return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomExperimentalDisabledErrMsg) - } if col.Tp.Tp != mysql.TypeLonglong { return ErrInvalidAutoRandom.GenWithStackByArgs( fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(col.Tp.Tp))) @@ -3031,7 +3027,8 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or return nil, errors.Trace(err) } - if err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil { + var newAutoRandBits uint64 + if newAutoRandBits, err = checkAutoRandom(t.Meta(), col, specNewColumn); err != nil { return nil, errors.Trace(err) } @@ -3041,7 +3038,7 @@ func (d *ddl) getModifiableColumnJob(ctx sessionctx.Context, ident ast.Ident, or SchemaName: schema.Name.L, Type: model.ActionModifyColumn, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{&newCol, originalColName, spec.Position, modifyColumnTp}, + Args: []interface{}{&newCol, originalColName, spec.Position, modifyColumnTp, newAutoRandBits}, } return job, nil } @@ -3083,34 +3080,43 @@ func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol return nil } -func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) error { - if !config.GetGlobalConfig().Experimental.AllowAutoRandom && containsColumnOption(specNewColumn, ast.ColumnOptionAutoRandom) { - return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomExperimentalDisabledErrMsg) - } - // Disallow add/drop/modify actions on auto_random. - newAutoRandomBit, err := extractAutoRandomBitsFromColDef(specNewColumn) +func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) (uint64, error) { + // Disallow add/drop actions on auto_random. + oldRandBits := tableInfo.AutoRandomBits + newRandBits, err := extractAutoRandomBitsFromColDef(specNewColumn) if err != nil { - return errors.Trace(err) - } - if tableInfo.AutoRandomBits != newAutoRandomBit { - return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) + return 0, errors.Trace(err) } - - if tableInfo.AutoRandomBits != 0 { + switch { + case oldRandBits == newRandBits: + break + case oldRandBits == 0 || newRandBits == 0: + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) + case autoid.MaxAutoRandomBits < newRandBits: + errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, + autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O) + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + case oldRandBits < newRandBits: + break // Increasing auto_random shard bits is allowed. + case oldRandBits > newRandBits: + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomDecreaseBitErrMsg) + } + + if oldRandBits != 0 { // Disallow changing the column field type. if originCol.Tp != specNewColumn.Tp.Tp { - return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg) + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg) } // Disallow changing auto_increment on auto_random column. if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) != mysql.HasAutoIncrementFlag(originCol.Flag) { - return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) } // Disallow specifying a default value on auto_random column. if containsColumnOption(specNewColumn, ast.ColumnOptionDefaultValue) { - return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithDefaultValueErrMsg) } } - return nil + return newRandBits, nil } // ChangeColumn renames an existing column and modifies the column's definition, diff --git a/ddl/serial_test.go b/ddl/serial_test.go index 90f709dac7946..b9d5726a943fc 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -830,6 +830,7 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { defer tk.MustExec("drop database if exists auto_random_db") tk.MustExec("use auto_random_db") tk.MustExec("drop table if exists t") + tk.MustExec("set @@allow_auto_random_explicit_insert = true") assertInvalidAutoRandomErr := func(sql string, errMsg string, args ...interface{}) { _, err := tk.Exec(sql) @@ -840,16 +841,19 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { assertPKIsNotHandle := func(sql, errCol string) { assertInvalidAutoRandomErr(sql, autoid.AutoRandomPKisNotHandleErrMsg, errCol) } - assertExperimentDisabled := func(sql string) { - assertInvalidAutoRandomErr(sql, autoid.AutoRandomExperimentalDisabledErrMsg) - } assertAlterValue := func(sql string) { assertInvalidAutoRandomErr(sql, autoid.AutoRandomAlterErrMsg) } + assertDecreaseBitErr := func(sql string) { + assertInvalidAutoRandomErr(sql, autoid.AutoRandomDecreaseBitErrMsg) + } assertWithAutoInc := func(sql string) { assertInvalidAutoRandomErr(sql, autoid.AutoRandomIncompatibleWithAutoIncErrMsg) } - assertOverflow := func(sql, colName string, autoRandBits uint64) { + assertOverflow := func(sql, colName string, maxAutoRandBits, actualAutoRandBits uint64) { + assertInvalidAutoRandomErr(sql, autoid.AutoRandomOverflowErrMsg, maxAutoRandBits, actualAutoRandBits, colName) + } + assertMaxOverflow := func(sql, colName string, autoRandBits uint64) { assertInvalidAutoRandomErr(sql, autoid.AutoRandomOverflowErrMsg, autoid.MaxAutoRandomBits, autoRandBits, colName) } assertModifyColType := func(sql string) { @@ -891,6 +895,13 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { assertPKIsNotHandle("create table t (a bigint auto_random(3), b bigint, primary key (a, b))", "a") assertPKIsNotHandle("create table t (a bigint auto_random(3), b int, c char, primary key (a, c))", "a") + // PKIsNotHandle: table is created when alter-primary-key = true. + config.GetGlobalConfig().AlterPrimaryKey = true + assertPKIsNotHandle("create table t (a bigint auto_random(3) primary key, b int)", "a") + assertPKIsNotHandle("create table t (a bigint auto_random(3) primary key, b int)", "a") + assertPKIsNotHandle("create table t (a int, b bigint auto_random(3) primary key)", "b") + config.GetGlobalConfig().AlterPrimaryKey = false + // Can not set auto_random along with auto_increment. assertWithAutoInc("create table t (a bigint auto_random(3) primary key auto_increment)") assertWithAutoInc("create table t (a bigint primary key auto_increment auto_random(3))") @@ -905,8 +916,12 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { }) // Overflow data type max length. - assertOverflow("create table t (a bigint auto_random(64) primary key)", "a", 64) - assertOverflow("create table t (a bigint auto_random(16) primary key)", "a", 16) + assertMaxOverflow("create table t (a bigint auto_random(64) primary key)", "a", 64) + assertMaxOverflow("create table t (a bigint auto_random(16) primary key)", "a", 16) + mustExecAndDrop("create table t (a bigint auto_random(5) primary key)", func() { + assertMaxOverflow("alter table t modify a bigint auto_random(64)", "a", 64) + assertMaxOverflow("alter table t modify a bigint auto_random(16)", "a", 16) + }) assertNonPositive("create table t (a bigint auto_random(0) primary key)") tk.MustGetErrMsg("create table t (a bigint auto_random(-1) primary key)", @@ -919,6 +934,13 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { mustExecAndDrop("create table t (a bigint primary key auto_random(4))") mustExecAndDrop("create table t (a bigint auto_random(4), primary key (a))") + // Increase auto_random bits. + mustExecAndDrop("create table t (a bigint auto_random(5) primary key)", func() { + tk.MustExec("alter table t modify a bigint auto_random(8)") + tk.MustExec("alter table t modify a bigint auto_random(10)") + tk.MustExec("alter table t modify a bigint auto_random(12)") + }) + // Auto_random can occur multiple times like other column attributes. mustExecAndDrop("create table t (a bigint auto_random(3) auto_random(2) primary key)") mustExecAndDrop("create table t (a bigint, b bigint auto_random(3) primary key auto_random(2))") @@ -935,9 +957,30 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { }) mustExecAndDrop("create table t (a bigint primary key)", func() { assertAlterValue("alter table t modify column a bigint auto_random(3)") - assertAlterValue("alter table t change column a b bigint auto_random(3)") }) + // Decrease auto_random bits is not allowed. + mustExecAndDrop("create table t (a bigint auto_random(10) primary key)", func() { + assertDecreaseBitErr("alter table t modify column a bigint auto_random(6)") + }) + mustExecAndDrop("create table t (a bigint auto_random(10) primary key)", func() { + assertDecreaseBitErr("alter table t modify column a bigint auto_random(1)") + }) + + originStep := autoid.GetStep() + autoid.SetStep(1) + // Increase auto_random bits but it will overlap with incremental bits. + mustExecAndDrop("create table t (a bigint unsigned auto_random(5) primary key)", func() { + const alterTryCnt, rebaseOffset = 3, 1 + insertSQL := fmt.Sprintf("insert into t values (%d)", ((1<<(64-10))-1)-rebaseOffset-alterTryCnt) + tk.MustExec(insertSQL) + // Try to rebase to 0..0011..1111 (54 `1`s). + tk.MustExec("alter table t modify a bigint unsigned auto_random(6)") + tk.MustExec("alter table t modify a bigint unsigned auto_random(10)") + assertOverflow("alter table t modify a bigint unsigned auto_random(11)", "a", 10, 11) + }) + autoid.SetStep(originStep) + // Modifying the field type of a auto_random column is not allowed. // Here the throw error is `ERROR 8200 (HY000): Unsupported modify column: length 11 is less than origin 20`, // instead of `ERROR 8216 (HY000): Invalid auto random: modifying the auto_random column type is not supported` @@ -977,10 +1020,6 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { tk.MustExec("insert into t values(3)") tk.MustExec("insert into t values()") }) - - // Disallow using it when allow-auto-random is not enabled. - config.GetGlobalConfig().Experimental.AllowAutoRandom = false - assertExperimentDisabled("create table auto_random_table (a int primary key auto_random(3))") } func (s *testSerialSuite) TestAutoRandomIncBitsIncrementAndOffset(c *C) { diff --git a/executor/insert_test.go b/executor/insert_test.go index 7a7798fff0a93..1f19a66cb82c5 100644 --- a/executor/insert_test.go +++ b/executor/insert_test.go @@ -21,12 +21,12 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/parser/terror" - "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/testkit" + "github.com/pingcap/tidb/util/testutil" ) func (s *testSuite8) TestInsertOnDuplicateKey(c *C) { @@ -968,13 +968,8 @@ type testSuite9 struct { } func (s *testSuite9) TestAutoRandomID(c *C) { - allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom - if !allowAutoRandom { - config.GetGlobalConfig().Experimental.AllowAutoRandom = true - defer func() { - config.GetGlobalConfig().Experimental.AllowAutoRandom = false - }() - } + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) @@ -1017,13 +1012,8 @@ func (s *testSuite9) TestAutoRandomID(c *C) { } func (s *testSuite9) TestMultiAutoRandomID(c *C) { - allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom - if !allowAutoRandom { - config.GetGlobalConfig().Experimental.AllowAutoRandom = true - defer func() { - config.GetGlobalConfig().Experimental.AllowAutoRandom = false - }() - } + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) @@ -1066,13 +1056,8 @@ func (s *testSuite9) TestMultiAutoRandomID(c *C) { } func (s *testSuite9) TestAutoRandomIDAllowZero(c *C) { - allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom - if !allowAutoRandom { - config.GetGlobalConfig().Experimental.AllowAutoRandom = true - defer func() { - config.GetGlobalConfig().Experimental.AllowAutoRandom = false - }() - } + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() tk := testkit.NewTestKit(c, s.store) tk.MustExec(`use test`) @@ -1104,13 +1089,8 @@ func (s *testSuite9) TestAutoRandomIDAllowZero(c *C) { } func (s *testSuite9) TestAutoRandomIDExplicit(c *C) { - allowAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom - if !allowAutoRandom { - config.GetGlobalConfig().Experimental.AllowAutoRandom = true - defer func() { - config.GetGlobalConfig().Experimental.AllowAutoRandom = false - }() - } + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() tk := testkit.NewTestKit(c, s.store) tk.MustExec("set @@allow_auto_random_explicit_insert = true") diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index d522efc660fd4..819450a2539c7 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -831,11 +831,8 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { r.Check(testkit.Rows("test1 tt id 41 AUTO_INCREMENT")) tk.MustExec("drop table tt") - oldAutoRandom := config.GetGlobalConfig().Experimental.AllowAutoRandom - config.GetGlobalConfig().Experimental.AllowAutoRandom = true - defer func() { - config.GetGlobalConfig().Experimental.AllowAutoRandom = oldAutoRandom - }() + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() tk.MustExec("set @@allow_auto_random_explicit_insert = true") // Test for a table with auto_random primary key. diff --git a/go.mod b/go.mod index 9396cc1f383a3..8da95bf97d9c1 100644 --- a/go.mod +++ b/go.mod @@ -24,18 +24,18 @@ require ( github.com/ngaut/sync2 v0.0.0-20141008032647-7a24ed77b2ef github.com/opentracing/basictracer-go v1.0.0 github.com/opentracing/opentracing-go v1.1.0 - github.com/pingcap/br v0.0.0-20200623060633-439a1c2b2bfd + github.com/pingcap/br v0.0.0-20200716021245-f1df51c11469 github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011 github.com/pingcap/failpoint v0.0.0-20200603062251-b230c36c413c github.com/pingcap/fn v0.0.0-20191016082858-07623b84a47d github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 - github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 + github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c github.com/pingcap/log v0.0.0-20200511115504-543df19646ad github.com/pingcap/parser v0.0.0-20200623164729-3a18f1e5dceb github.com/pingcap/pd/v4 v4.0.0-rc.2.0.20200520083007-2c251bd8f181 github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1 - github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible + github.com/pingcap/tidb-tools v4.0.0-rc.2.0.20200521050818-6dd445d83fe0+incompatible github.com/pingcap/tipb v0.0.0-20200522051215-f31a15d98fce github.com/prometheus/client_golang v1.5.1 github.com/prometheus/client_model v0.2.0 @@ -55,10 +55,9 @@ require ( golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd golang.org/x/text v0.3.2 - golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1 + golang.org/x/tools v0.0.0-20200521211927-2b542361a4fc google.golang.org/grpc v1.26.0 gopkg.in/natefinch/lumberjack.v2 v2.0.0 - honnef.co/go/tools v0.0.1-2020.1.4 // indirect sourcegraph.com/sourcegraph/appdash v0.0.0-20180531100431-4c381bd170b4 sourcegraph.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67 ) diff --git a/go.sum b/go.sum index b5c6e43c6a7dd..b39bb1d4787dc 100644 --- a/go.sum +++ b/go.sum @@ -79,6 +79,7 @@ github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmf github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20181031085051-9002847aa142/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= +github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e h1:Wf6HqHfScWJN9/ZjdUKyjop4mf3Qdd+1TvvltAvM3m8= github.com/coreos/go-systemd v0.0.0-20190321100706-95778dfbb74e/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f h1:JOrtw2xFKzlg+cbHpyrpLDmnN1HqhBfnX7WDiW7eG2c= github.com/coreos/go-systemd v0.0.0-20190719114852-fd7a80b32e1f/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= @@ -335,6 +336,7 @@ github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJ github.com/modern-go/reflect2 v0.0.0-20180701023420-4b7aa43c6742/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= github.com/modern-go/reflect2 v1.0.1 h1:9f412s+6RmYXLWZSEzVVgPGK7C2PphHj5RJrvfx9AWI= github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/montanaflynn/stats v0.0.0-20151014174947-eeaced052adb h1:bsjNADsjHq0gjU7KO7zwoX5k3HtFdf6TDzB3ncl5iUs= github.com/montanaflynn/stats v0.0.0-20151014174947-eeaced052adb/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/montanaflynn/stats v0.0.0-20180911141734-db72e6cae808/go.mod h1:wL8QJuTMNUDYhXwkmfOly8iTdp5TEcJFWZD2D7SIkUc= github.com/montanaflynn/stats v0.5.0 h1:2EkzeTSqBB4V4bJwWrt5gIIrZmpJBcoIRGS2kWLgzmk= @@ -349,10 +351,12 @@ github.com/nicksnyder/go-i18n v1.10.0/go.mod h1:HrK7VCrbOvQoUAQ7Vpy7i87N7JZZZ7R2 github.com/oklog/ulid v1.3.1/go.mod h1:CirwcVhetQ6Lv90oh/F+FBtV6XMibvdAFo93nm5qn4U= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= +github.com/onsi/ginkgo v1.6.0 h1:Ix8l273rp3QzYgXSR+c8d1fTG7UPgYkOSELPhiY/YGw= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.11.0 h1:JAKSXpt1YjtLA7YpPiqO9ss6sNXEsPfSGdwN0UHqzrw= github.com/onsi/ginkgo v1.11.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/gomega v1.4.2 h1:3mYCb7aPxS/RU7TI1y4rkEn1oKmPRjNJLNEXgw7MH2I= github.com/onsi/gomega v1.4.2/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/onsi/gomega v1.8.1 h1:C5Dqfs/LeauYDX0jJXIe2SWmwCbGzx9yF8C8xy3Lh34= @@ -370,10 +374,11 @@ github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5 github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200407064406-b2b8ad403d01/go.mod h1:77fCh8d3oKzC5ceOJWeZXAS/mLzVgdZ7rKniwmOyFuo= github.com/pingcap-incubator/tidb-dashboard v0.0.0-20200514075710-eecc9a4525b5/go.mod h1:8q+yDx0STBPri8xS4A2duS1dAf+xO0cMtjwe0t6MWJk= github.com/pingcap/br v0.0.0-20200426093517-dd11ae28b885/go.mod h1:4w3meMnk7HDNpNgjuRAxavruTeKJvUiXxoEWTjzXPnA= -github.com/pingcap/br v0.0.0-20200521085655-53201addd4ad h1:nptiQT0kWdIUghh49OyaTBYb4DtdxJmsLHOxbU25kW4= github.com/pingcap/br v0.0.0-20200521085655-53201addd4ad/go.mod h1:SlSUHWY7QUoooiYxOKuJ8kUh2KjI29ogBh89YXz2dLA= github.com/pingcap/br v0.0.0-20200623060633-439a1c2b2bfd h1:vEoTsslkTbSiMMAY8XHsI/D0gih8y/kOPQytXYgc7t0= github.com/pingcap/br v0.0.0-20200623060633-439a1c2b2bfd/go.mod h1:NGee2H9vXLunFIBXGb3uFsWRpw3BBo822sY4dyXepqo= +github.com/pingcap/br v0.0.0-20200716021245-f1df51c11469 h1:sp6f6H8j9Iqt1rVzA1XOs4oCEumb9S3lL3L5UqLpKWo= +github.com/pingcap/br v0.0.0-20200716021245-f1df51c11469/go.mod h1:Ft2Vuvj6XJkbjQvflDOesJTy+9bui0saz0UonIgipAw= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= @@ -403,6 +408,8 @@ github.com/pingcap/kvproto v0.0.0-20200423020121-038e31959c2a/go.mod h1:IOdRDPLy github.com/pingcap/kvproto v0.0.0-20200424032552-6650270c39c3/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29 h1:NpW1OuYrIl+IQrSsVbtyHpHpazmSCHy+ysrOixY0xY4= github.com/pingcap/kvproto v0.0.0-20200518112156-d4aeb467de29/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= +github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c h1:VnLpCAxMAeDxc7HXTetwDQB+/MtDQjHAOBsd4QnGVwA= +github.com/pingcap/kvproto v0.0.0-20200706115936-1e0910aabe6c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/log v0.0.0-20191012051959-b742a5d432e9/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200117041106-d28c14d3b1cd/go.mod h1:4rbK1p9ILyIfb6hU7OG2CiWSqMXnp3JMbiaVJ6mvoY8= github.com/pingcap/log v0.0.0-20200511115504-543df19646ad h1:SveG82rmu/GFxYanffxsSF503SiQV+2JLnWEiGiF+Tc= @@ -421,10 +428,13 @@ github.com/pingcap/sysutil v0.0.0-20200408114249-ed3bd6f7fdb1/go.mod h1:EB/852NM github.com/pingcap/tidb v1.1.0-beta.0.20200424154252-5ede18f10eed/go.mod h1:m2VDlJDbUeHPCXAfKPajqLmB1uLvWpkKk3zALNqDYdw= github.com/pingcap/tidb v1.1.0-beta.0.20200509133407-a9dc72cf2558/go.mod h1:cXNbVSQAkwwmjFQmEnEPI00Z2/Y/KOhouttUPERiInE= github.com/pingcap/tidb v1.1.0-beta.0.20200606093724-b5b4da0e6a90/go.mod h1:aaBBi3OJmYjENWY31YYOY8K6UoZZYgjZVZH56D0QIdE= +github.com/pingcap/tidb v1.1.0-beta.0.20200715100003-b4da443a3c4c/go.mod h1:TplKBs1sevRvK11aT7ro0ntTCalyh1fMaWACp03dQf4= github.com/pingcap/tidb-tools v4.0.0-beta.1.0.20200306084441-875bd09aa3d5+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200421113014-507d2bb3a15e+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible h1:/JKsYjsa5Ug8v5CN4zIbJGIqsvgBUkGwaP/rEScVvWM= github.com/pingcap/tidb-tools v4.0.0-rc.1.0.20200514040632-f76b3e428e19+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= +github.com/pingcap/tidb-tools v4.0.0-rc.2.0.20200521050818-6dd445d83fe0+incompatible h1:e+j+rsJYX+J7eTkgjnGBH2/T3NS6GNSPD6nHA5bPdCI= +github.com/pingcap/tidb-tools v4.0.0-rc.2.0.20200521050818-6dd445d83fe0+incompatible/go.mod h1:XGdcy9+yqlDSEMTpOXnwf3hiTeqrV6MN/u1se9N8yIM= github.com/pingcap/tipb v0.0.0-20190428032612-535e1abaa330/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200417094153-7316d94df1ee/go.mod h1:RtkHW8WbcNxj8lsbzjaILci01CtYnYbIkQhjyZWrWVI= github.com/pingcap/tipb v0.0.0-20200522051215-f31a15d98fce h1:LDyY6Xh/Z/SHVQ10erWtoOwIxHSTtlpPQ9cvS+BfRMY= @@ -514,6 +524,7 @@ github.com/swaggo/swag v1.5.1/go.mod h1:1Bl9F/ZBpVWh22nY0zmYyASPO1lI/zIwRDrpZU+t github.com/swaggo/swag v1.6.3/go.mod h1:wcc83tB4Mb2aNiL/HP4MFeQdpHUrca+Rp/DRNgWAUio= github.com/swaggo/swag v1.6.5/go.mod h1:Y7ZLSS0d0DdxhWGVhQdu+Bu1QhaF5k0RD7FKdiAykeY= github.com/swaggo/swag v1.6.6-0.20200323071853-8e21f4cefeea/go.mod h1:xDhTyuFIujYiN3DKWC/H/83xcfHp+UE/IzWWampG7Zc= +github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d h1:4J9HCZVpvDmj2tiKGSTUnb3Ok/9CEQb9oqu9LHKQQpc= github.com/syndtr/goleveldb v0.0.0-20180815032940-ae2bd5eed72d/go.mod h1:Z4AUp2Km+PwemOoO/VB5AOx9XSsIItzFjoJlOSiYmn0= github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285 h1:uSDYjYejelKyceA6DiCsngFof9jAyeaSyX9XC5a1a7Q= github.com/syndtr/goleveldb v1.0.1-0.20190625010220-02440ea7a285/go.mod h1:9OrXJhf154huy1nPWmuSrkgjPUtUNhA+Zmy+6AESzuA= @@ -551,6 +562,7 @@ github.com/xiang90/probing v0.0.0-20190116061207-43a291ad63a2/go.mod h1:UETIi67q github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= github.com/yookoala/realpath v1.0.0/go.mod h1:gJJMA9wuX7AcqLy1+ffPatSCySA1FQ2S8Ya9AIoYBpE= github.com/yuin/goldmark v1.1.25/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.3 h1:MUGmc65QhB3pIlaQ5bB4LwqSj6GIonVJXpZiaKNyaKk= go.etcd.io/bbolt v1.3.3/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= @@ -617,6 +629,7 @@ golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHl golang.org/x/lint v0.0.0-20190409202823-959b441ac422/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190909230951-414d861bb4ac/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20190930215403-16217165b5de/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f h1:J5lckAjkw6qYlOZNj90mLYNTEKDvWeuc1yieZ8qUzUE= golang.org/x/lint v0.0.0-20191125180803-fdd1cda4f05f/go.mod h1:5qLYkcX4OjUUV8bRuDixDT3tpyyb+LUpUlRWLxfhWrs= golang.org/x/lint v0.0.0-20200302205851-738671d3881b h1:Wh+f8QHJXR411sJR8/vRBTZ7YapZaRvUcLFFJhusH0k= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= @@ -739,6 +752,8 @@ golang.org/x/tools v0.0.0-20200225230052-807dcd883420/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200325010219-a49f79bcc224/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1 h1:odiryKYJy7CjdrZxhrcE1Z8L9+kGyGZOnfpuauvdCeU= golang.org/x/tools v0.0.0-20200325203130-f53864d0dba1/go.mod h1:Sl4aGygMT6LrqrWclx+PTx3U+LnKx/seiNR+3G19Ar8= +golang.org/x/tools v0.0.0-20200521211927-2b542361a4fc h1:6m2YO+AmBApbUOmhsghW+IfRyZOY4My4UYvQQrEpHfY= +golang.org/x/tools v0.0.0-20200521211927-2b542361a4fc/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543 h1:E7g+9GITq07hpfrRu66IVDexMakfv52eLZ2CXBWiKr4= diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index 234db6c6bdf0e..3f92ecc023250 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -30,19 +30,19 @@ var ( const ( // AutoRandomPKisNotHandleErrMsg indicates the auto_random column attribute is defined on a non-primary key column, or the table's primary key is not a single integer column. - AutoRandomPKisNotHandleErrMsg = "column %s is not the single integer primary key, or alter-primary-key is enabled" - // AutoRandomExperimentalDisabledErrMsg is reported when the experimental option allow-auto-random is not enabled. - AutoRandomExperimentalDisabledErrMsg = "auto_random is an experimental feature, which can only be used when allow-auto-random is enabled. This can be changed in the configuration." + AutoRandomPKisNotHandleErrMsg = "column %s is not the integer primary key, or table is created with alter-primary-key enabled" // AutoRandomIncompatibleWithAutoIncErrMsg is reported when auto_random and auto_increment are specified on the same column. AutoRandomIncompatibleWithAutoIncErrMsg = "auto_random is incompatible with auto_increment" // AutoRandomIncompatibleWithDefaultValueErrMsg is reported when auto_random and default are specified on the same column. AutoRandomIncompatibleWithDefaultValueErrMsg = "auto_random is incompatible with default" // AutoRandomOverflowErrMsg is reported when auto_random is greater than max length of a MySQL data type. - AutoRandomOverflowErrMsg = "max allowed auto_random bits is %d, but got %d on column `%s`" + AutoRandomOverflowErrMsg = "max allowed auto_random shard bits is %d, but got %d on column `%s`" // AutoRandomModifyColTypeErrMsg is reported when a user is trying to modify the type of a column specified with auto_random. AutoRandomModifyColTypeErrMsg = "modifying the auto_random column type is not supported" // AutoRandomAlterErrMsg is reported when a user is trying to add/drop/modify the value of auto_random attribute. AutoRandomAlterErrMsg = "adding/dropping/modifying auto_random is not supported" + // AutoRandomDecreaseBitErrMsg is reported when the auto_random shard bits is decreased. + AutoRandomDecreaseBitErrMsg = "decreasing auto_random shard bits is not supported" // AutoRandomNonPositive is reported then a user specifies a non-positive value for auto_random. AutoRandomNonPositive = "the value of auto_random should be positive" // AutoRandomAvailableAllocTimesNote is reported when a table containing auto_random is created. diff --git a/meta/meta.go b/meta/meta.go index 6514ffd96f726..fcc1e63be40f2 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -212,7 +212,7 @@ func (m *Meta) GetAutoTableID(dbID int64, tableID int64) (int64, error) { return m.txn.HGetInt64(m.dbKey(dbID), m.autoTableIDKey(tableID)) } -// GetAutoRandomID gets current auto shard id with table id. +// GetAutoRandomID gets current auto random id with table id. func (m *Meta) GetAutoRandomID(dbID int64, tableID int64) (int64, error) { return m.txn.HGetInt64(m.dbKey(dbID), m.autoRandomTableIDKey(tableID)) } diff --git a/util/testutil/testutil.go b/util/testutil/testutil.go index 7c3f621a5e358..8e962fb6f7d72 100644 --- a/util/testutil/testutil.go +++ b/util/testutil/testutil.go @@ -319,21 +319,18 @@ type autoRandom struct { originAlterPrimaryKey bool } -// SetupAutoRandomTestConfig set alter-primary-key to false, and set allow-auto-random to true and save their origin values. +// SetupAutoRandomTestConfig set alter-primary-key to false and save its origin values. // This method should only be used for the tests in SerialSuite. func (a *autoRandom) SetupAutoRandomTestConfig() { globalCfg := config.GetGlobalConfig() - a.originAllowAutoRandom = globalCfg.Experimental.AllowAutoRandom a.originAlterPrimaryKey = globalCfg.AlterPrimaryKey globalCfg.AlterPrimaryKey = false - globalCfg.Experimental.AllowAutoRandom = true } // RestoreAutoRandomTestConfig restore the values had been saved in SetupTestConfig. // This method should only be used for the tests in SerialSuite. func (a *autoRandom) RestoreAutoRandomTestConfig() { globalCfg := config.GetGlobalConfig() - globalCfg.Experimental.AllowAutoRandom = a.originAllowAutoRandom globalCfg.AlterPrimaryKey = a.originAlterPrimaryKey }