From 9b2443ab3173457f5e83e2c2acfcbfe1351f8bab Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Thu, 13 May 2021 17:19:40 +0800 Subject: [PATCH] ddl: support change from auto_inc to auto_random through 'alter table' (#20512) (#24608) --- ddl/column.go | 82 ++++++++++++++++++++------- ddl/db_integration_test.go | 14 +++++ ddl/ddl_api.go | 79 ++++++++++++++------------ ddl/ddl_worker.go | 2 +- ddl/serial_test.go | 65 ++++++++++++++++++++- ddl/table.go | 13 +++-- executor/seqtest/seq_executor_test.go | 6 +- go.sum | 20 ------- infoschema/builder.go | 30 ++++++---- meta/autoid/autoid.go | 79 +++++++++++++++++++------- meta/autoid/errors.go | 6 ++ meta/meta.go | 9 +++ 12 files changed, 281 insertions(+), 124 deletions(-) diff --git a/ddl/column.go b/ddl/column.go index 42e5de30efa4f..26a28a625eaf8 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -21,7 +21,6 @@ import ( "sync/atomic" "time" - "github.com/cznic/mathutil" "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/parser/ast" @@ -353,7 +352,7 @@ func onSetDefaultValue(t *meta.Meta, job *model.Job) (ver int64, _ error) { return updateColumnDefaultValue(t, job, newCol, &newCol.Name) } -func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) { +func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { newCol := &model.ColumnInfo{} oldColName := &model.CIStr{} pos := &ast.ColumnPosition{} @@ -365,12 +364,12 @@ func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, errors.Trace(err) } - return w.doModifyColumn(t, job, newCol, oldColName, pos, modifyColumnTp, updatedAutoRandomBits) + return w.doModifyColumn(d, 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, + d *ddlCtx, 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 { @@ -413,11 +412,12 @@ func (w *worker) doModifyColumn( } }) - if newAutoRandBits > 0 { - if err := checkAndApplyNewAutoRandomBits(job, t, tblInfo, newCol, oldName, newAutoRandBits); err != nil { - return ver, errors.Trace(err) - } + err = checkAndApplyAutoRandomBits(d, t, dbInfo, tblInfo, oldCol, newCol, newAutoRandBits) + if err != nil { + job.State = model.JobStateCancelled + 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) @@ -508,31 +508,73 @@ func (w *worker) doModifyColumn( 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 +func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, tblInfo *model.TableInfo, + oldCol *model.ColumnInfo, newCol *model.ColumnInfo, newAutoRandBits uint64) error { + if newAutoRandBits == 0 { + return nil + } + err := checkNewAutoRandomBits(m, dbInfo.ID, tblInfo.ID, oldCol, newCol, newAutoRandBits) + if err != nil { + return err + } + return applyNewAutoRandomBits(d, m, dbInfo, tblInfo, oldCol, newAutoRandBits) +} + +// checkNewAutoRandomBits checks whether the new auto_random bits number can cause overflow. +func checkNewAutoRandomBits(m *meta.Meta, schemaID, tblID int64, + oldCol *model.ColumnInfo, newCol *model.ColumnInfo, newAutoRandBits uint64) error { newLayout := autoid.NewShardIDLayout(&newCol.FieldType, newAutoRandBits) - // GenAutoRandomID first to prevent concurrent update. - _, err := t.GenAutoRandomID(schemaID, tblInfo.ID, 1) + allocTp := autoid.AutoRandomType + convertedFromAutoInc := mysql.HasAutoIncrementFlag(oldCol.Flag) + if convertedFromAutoInc { + allocTp = autoid.AutoIncrementType + } + // GenerateAutoID first to prevent concurrent update in DML. + _, err := autoid.GenerateAutoID(m, schemaID, tblID, 1, allocTp) if err != nil { return err } - currentIncBitsVal, err := t.GetAutoRandomID(schemaID, tblInfo.ID) + currentIncBitsVal, err := autoid.GetAutoID(m, schemaID, tblID, allocTp) 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 + usedBits := uint64(64 - bits.LeadingZeros64(uint64(currentIncBitsVal))) + if usedBits > newLayout.IncrementalBits { + overflowCnt := usedBits - newLayout.IncrementalBits + errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, newAutoRandBits-overflowCnt, newAutoRandBits, oldCol.Name.O) return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) } + return nil +} + +// applyNewAutoRandomBits set auto_random bits to TableInfo and +// migrate auto_increment ID to auto_random ID if possible. +func applyNewAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, + tblInfo *model.TableInfo, oldCol *model.ColumnInfo, newAutoRandBits uint64) error { tblInfo.AutoRandomBits = newAutoRandBits + needMigrateFromAutoIncToAutoRand := mysql.HasAutoIncrementFlag(oldCol.Flag) + if !needMigrateFromAutoIncToAutoRand { + return nil + } + autoRandAlloc := autoid.NewAllocatorsFromTblInfo(d.store, dbInfo.ID, tblInfo).Get(autoid.AutoRandomType) + if autoRandAlloc == nil { + errMsg := fmt.Sprintf(autoid.AutoRandomAllocatorNotFound, dbInfo.Name.O, tblInfo.Name.O) + return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } + nextAutoIncID, err := m.GetAutoTableID(dbInfo.ID, tblInfo.ID) + if err != nil { + return errors.Trace(err) + } + err = autoRandAlloc.Rebase(tblInfo.ID, nextAutoIncID, false) + if err != nil { + return errors.Trace(err) + } + if err := m.CleanAutoID(dbInfo.ID, tblInfo.ID); err != nil { + return errors.Trace(err) + } return nil } diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index f9c25d0689a9e..00abae736206d 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2348,3 +2348,17 @@ func (s *testIntegrationSuite7) TestDuplicateErrorMessage(c *C) { } } } + +func (s *testIntegrationSuite7) TestAutoIncrementAllocator(c *C) { + tk := testkit.NewTestKit(c, s.store) + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.AlterPrimaryKey = false + }) + tk.MustExec("drop database if exists test_create_table_option_auto_inc;") + tk.MustExec("create database test_create_table_option_auto_inc;") + tk.MustExec("use test_create_table_option_auto_inc;") + + tk.MustExec("create table t (a bigint primary key) auto_increment = 10;") + tk.MustExec("alter table t auto_increment = 10;") +} diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 5e12df0920af8..8edac33dd847d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1574,14 +1574,16 @@ func (d *ddl) CreateTableWithInfo( if tbInfo.AutoIncID > 1 { // Default tableAutoIncID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. - if err = d.handleAutoIncID(tbInfo, schema.ID, autoid.RowIDAllocType); err != nil { + newEnd := tbInfo.AutoIncID - 1 + if err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType); err != nil { return errors.Trace(err) } } if tbInfo.AutoRandID > 1 { // Default tableAutoRandID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. - err = d.handleAutoIncID(tbInfo, schema.ID, autoid.AutoRandomType) + newEnd := tbInfo.AutoRandID - 1 + err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoRandomType) } } @@ -1884,22 +1886,11 @@ func checkCharsetAndCollation(cs string, co string) error { // handleAutoIncID handles auto_increment option in DDL. It creates a ID counter for the table and initiates the counter to a proper value. // For example if the option sets auto_increment to 10. The counter will be set to 9. So the next allocated ID will be 10. -func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, tp autoid.AllocatorType) error { +func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, newEnd int64, tp autoid.AllocatorType) error { allocs := autoid.NewAllocatorsFromTblInfo(d.store, schemaID, tbInfo) - tbInfo.State = model.StatePublic - tb, err := table.TableFromMeta(allocs, tbInfo) - if err != nil { - return errors.Trace(err) - } - // The operation of the minus 1 to make sure that the current value doesn't be used, - // the next Alloc operation will get this value. - // Its behavior is consistent with MySQL. - if tp == autoid.RowIDAllocType { - if err = tb.RebaseAutoID(nil, tbInfo.AutoIncID-1, false, tp); err != nil { - return errors.Trace(err) - } - } else { - if err = tb.RebaseAutoID(nil, tbInfo.AutoRandID-1, false, tp); err != nil { + if alloc := allocs.Get(tp); alloc != nil { + err := alloc.Rebase(tbInfo.ID, newEnd, false) + if err != nil { return errors.Trace(err) } } @@ -2240,16 +2231,18 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 actionType = model.ActionRebaseAutoID } - autoID, err := t.Allocators(ctx).Get(tp).NextGlobalAutoID(t.Meta().ID) - if err != nil { - return errors.Trace(err) + if alloc := t.Allocators(ctx).Get(tp); alloc != nil { + autoID, err := alloc.NextGlobalAutoID(t.Meta().ID) + if err != nil { + return errors.Trace(err) + } + // If newBase < autoID, we need to do a rebase before returning. + // Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000. + // If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B, + // and TiDB-B finds 100 < 30001 but returns without any handling, + // then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user. + newBase = int64(mathutil.MaxUint64(uint64(newBase), uint64(autoID))) } - // If newBase < autoID, we need to do a rebase before returning. - // Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000. - // If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B, - // and TiDB-B finds 100 < 30001 but returns without any handling, - // then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user. - newBase = int64(mathutil.MaxUint64(uint64(newBase), uint64(autoID))) job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, @@ -3119,9 +3112,8 @@ func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol } func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) (uint64, error) { - // Disallow add/drop actions on auto_random. var oldRandBits uint64 - if tableInfo.PKIsHandle && (tableInfo.GetPkName().L == originCol.Name.L) { + if originCol.IsPKHandleColumn(tableInfo) { oldRandBits = tableInfo.AutoRandomBits } newRandBits, err := extractAutoRandomBitsFromColDef(specNewColumn) @@ -3131,25 +3123,38 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe 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: + addingAutoRandom := oldRandBits == 0 + if addingAutoRandom { + convFromAutoInc := mysql.HasAutoIncrementFlag(originCol.Flag) && originCol.IsPKHandleColumn(tableInfo) + if !convFromAutoInc { + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterChangeFromAutoInc) + } + } + if autoid.MaxAutoRandomBits < newRandBits { + errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, + autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O) + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg) + } break // Increasing auto_random shard bits is allowed. case oldRandBits > newRandBits: + if newRandBits == 0 { + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg) + } return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomDecreaseBitErrMsg) } - if oldRandBits != 0 { + modifyingAutoRandCol := oldRandBits > 0 || newRandBits > 0 + if modifyingAutoRandCol { // Disallow changing the column field type. if originCol.Tp != specNewColumn.Tp.Tp { return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg) } - // Disallow changing auto_increment on auto_random column. - if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) != mysql.HasAutoIncrementFlag(originCol.Flag) { + if originCol.Tp != mysql.TypeLonglong { + return 0, ErrInvalidAutoRandom.GenWithStackByArgs(fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(originCol.Tp))) + } + // Disallow changing from auto_random to auto_increment column. + if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) { return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg) } // Disallow specifying a default value on auto_random column. diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 0980a0e159391..7b18e98c021ee 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -689,7 +689,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionDropColumn: ver, err = onDropColumn(t, job) case model.ActionModifyColumn: - ver, err = w.onModifyColumn(t, job) + ver, err = w.onModifyColumn(d, t, job) case model.ActionSetDefaultValue: ver, err = onSetDefaultValue(t, job) case model.ActionAddIndex: diff --git a/ddl/serial_test.go b/ddl/serial_test.go index ea2e94f347873..487b5626209cf 100644 --- a/ddl/serial_test.go +++ b/ddl/serial_test.go @@ -835,6 +835,9 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { assertAlterValue := func(sql string) { assertInvalidAutoRandomErr(sql, autoid.AutoRandomAlterErrMsg) } + assertOnlyChangeFromAutoIncPK := func(sql string) { + assertInvalidAutoRandomErr(sql, autoid.AutoRandomAlterChangeFromAutoInc) + } assertDecreaseBitErr := func(sql string) { assertInvalidAutoRandomErr(sql, autoid.AutoRandomDecreaseBitErrMsg) } @@ -952,11 +955,11 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { assertAlterValue("alter table t change column c d bigint") }) mustExecAndDrop("create table t (a bigint primary key)", func() { - assertAlterValue("alter table t modify column a bigint auto_random(3)") + assertOnlyChangeFromAutoIncPK("alter table t modify column a bigint auto_random(3)") }) mustExecAndDrop("create table t (a bigint, b bigint, primary key(a, b))", func() { - assertAlterValue("alter table t modify column a bigint auto_random(3)") - assertAlterValue("alter table t modify column b bigint auto_random(3)") + assertOnlyChangeFromAutoIncPK("alter table t modify column a bigint auto_random(3)") + assertOnlyChangeFromAutoIncPK("alter table t modify column b bigint auto_random(3)") }) // Decrease auto_random bits is not allowed. @@ -1025,6 +1028,62 @@ func (s *testSerialSuite) TestAutoRandom(c *C) { }) } +func (s *testSerialSuite) TestAutoRandomChangeFromAutoInc(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test;") + tk.MustExec("set @@tidb_allow_remove_auto_inc = 1;") + + // Basic usages. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint auto_increment primary key);") + tk.MustExec("insert into t values (), (), ();") + tk.MustExec("alter table t modify column a bigint auto_random(3);") + tk.MustExec("insert into t values (), (), ();") + rows := tk.MustQuery("show table t next_row_id;").Rows() + c.Assert(len(rows), Equals, 1, Commentf("query result: %v", rows)) + c.Assert(len(rows[0]), Equals, 5, Commentf("query result: %v", rows)) + c.Assert(rows[0][4], Equals, "AUTO_RANDOM") + + // Changing from auto_inc unique key is not allowed. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint auto_increment unique key);") + tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint auto_increment unique key, b bigint auto_random primary key);") + tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom) + + // Changing from non-auto-inc column is not allowed. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint);") + tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint primary key);") + tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom) + + // Changing from non BIGINT auto_inc pk column is not allowed. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int auto_increment primary key);") + tk.MustGetErrCode("alter table t modify column a int auto_random;", errno.ErrInvalidAutoRandom) + tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom) + + // Changing from auto_random to auto_increment is not allowed. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint auto_random primary key);") + // "Unsupported modify column: can't set auto_increment" + tk.MustGetErrCode("alter table t modify column a bigint auto_increment;", errno.ErrUnsupportedDDLOperation) + + // Large auto_increment number overflows auto_random. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint auto_increment primary key);") + tk.MustExec("insert into t values (1<<(64-5));") + // "max allowed auto_random shard bits is 3, but got 4 on column `a`" + tk.MustGetErrCode("alter table t modify column a bigint auto_random(4);", errno.ErrInvalidAutoRandom) + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint auto_increment primary key);") + tk.MustExec("insert into t values (1<<(64-6));") + tk.MustExec("alter table t modify column a bigint auto_random(4);") +} + func (s *testSerialSuite) TestAutoRandomIncBitsIncrementAndOffset(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("create database if not exists auto_random_db") diff --git a/ddl/table.go b/ddl/table.go index 96569e59833ee..6ec6e18b69df6 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -542,12 +542,13 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.Al job.State = model.JobStateCancelled return ver, errors.Trace(err) } - // The operation of the minus 1 to make sure that the current value doesn't be used, - // the next Alloc operation will get this value. - // Its behavior is consistent with MySQL. - err = tbl.RebaseAutoID(nil, newBase-1, false, tp) - if err != nil { - return ver, errors.Trace(err) + if alloc := tbl.Allocators(nil).Get(tp); alloc != nil { + // The next value to allocate is `newBase`. + newEnd := newBase - 1 + err = alloc.Rebase(tblInfo.ID, newEnd, false) + if err != nil { + return ver, errors.Trace(err) + } } ver, err = updateVersionAndTableInfo(t, job, tblInfo, true) if err != nil { diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 8660cc175500b..738a54122e1ba 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -842,15 +842,15 @@ func HelperTestAdminShowNextID(c *C, s *seqTestSuite, str string) { tk.MustExec("create table t3(id bigint primary key auto_random(5), c int)") // Start handle is 1. r = tk.MustQuery(str + " t3 next_row_id") - r.Check(testkit.Rows("test1 t3 _tidb_rowid 1 AUTO_INCREMENT", "test1 t3 id 1 AUTO_RANDOM")) + r.Check(testkit.Rows("test1 t3 id 1 AUTO_RANDOM")) // Insert some rows. tk.MustExec("insert into t3 (c) values (1), (2);") r = tk.MustQuery(str + " t3 next_row_id") - r.Check(testkit.Rows("test1 t3 _tidb_rowid 1 AUTO_INCREMENT", "test1 t3 id 11 AUTO_RANDOM")) + r.Check(testkit.Rows("test1 t3 id 11 AUTO_RANDOM")) // Rebase. tk.MustExec("insert into t3 (id, c) values (103, 3);") r = tk.MustQuery(str + " t3 next_row_id") - r.Check(testkit.Rows("test1 t3 _tidb_rowid 1 AUTO_INCREMENT", "test1 t3 id 114 AUTO_RANDOM")) + r.Check(testkit.Rows("test1 t3 id 114 AUTO_RANDOM")) // Test for a sequence. tk.MustExec("create sequence seq1 start 15 cache 57") diff --git a/go.sum b/go.sum index 0b6f407956713..62761b705e09e 100644 --- a/go.sum +++ b/go.sum @@ -25,14 +25,11 @@ github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03 github.com/BurntSushi/xgb v0.0.0-20160522181843-27f122750802/go.mod h1:IVnqGOEym/WlBOVXweHU+Q+/VP0lqqI8lqeDx9IjBqo= github.com/Jeffail/gabs/v2 v2.5.1 h1:ANfZYjpMlfTTKebycu4X1AgkVWumFVDYQl7JwOr4mDk= github.com/Jeffail/gabs/v2 v2.5.1/go.mod h1:xCn81vdHKxFUuWWAaD5jCTQDNPBMh5pPs9IJ+NcziBI= -github.com/KyleBanks/depth v1.2.1 h1:5h8fQADFrWtarTdtDudMmGsC7GPbOAu6RVB3ffsVFHc= github.com/KyleBanks/depth v1.2.1/go.mod h1:jzSb9d0L43HxTQfT+oSA1EEp2q+ne2uh6XgeJcm8brE= github.com/OneOfOne/xxhash v1.2.2 h1:KMrpdQIwFcEqXDklaen+P1axHaj9BSKzvpUUfnHldSE= github.com/OneOfOne/xxhash v1.2.2/go.mod h1:HSdplMjZKSmBqAxg5vPj2TmRDmfkzw+cTzAElWljhcU= github.com/PuerkitoBio/purell v1.1.0/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/purell v1.1.1 h1:WEQqlqaGbrPkxLJWfBwQmfEAE1Z7ONdDLqrN38tNFfI= github.com/PuerkitoBio/purell v1.1.1/go.mod h1:c11w/QuzBsJSee3cPx9rAFu61PvFxuPbtSwDGJws/X0= -github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578 h1:d+Bc7a5rLufV/sSk/8dngufqelfh6jnri85riMAaF/M= github.com/PuerkitoBio/urlesc v0.0.0-20170810143723-de5bf2ad4578/go.mod h1:uGdkoq3SwY9Y+13GIhn11/XLaGBb4BfwItxLd5jeuXE= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d h1:G0m3OIz70MZUWq3EgK3CesDbo8upS2Vm9/P3FtgI+Jk= github.com/StackExchange/wmi v0.0.0-20190523213315-cbe66965904d/go.mod h1:3eOhrUMpNV+6aFIbp5/iudMxNCF27Vw2OZgy4xEx0Fg= @@ -90,7 +87,6 @@ github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f h1:lBNOc5arjvs8E5mO2tbp github.com/coreos/pkg v0.0.0-20180928190104-399ea9e2e55f/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA= github.com/corona10/goimagehash v1.0.2/go.mod h1:/l9umBhvcHQXVtQO1V6Gp1yD20STawkhRnnX0D1bvVI= github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= -github.com/cpuguy83/go-md2man/v2 v2.0.0 h1:EoUDS0afbrsXAZ9YQ9jdu/mZ2sXgT1/2yyNng4PGlyM= github.com/cpuguy83/go-md2man/v2 v2.0.0/go.mod h1:maD7wRr/U5Z6m/iR4s+kqSMx2CaBsrgA7czyZG/E6dU= github.com/creack/pty v1.1.7/go.mod h1:lj5s0c3V2DBrqTV7llrYr5NG6My20zk30Fl46Y7DoTY= github.com/cznic/golex v0.0.0-20181122101858-9c343928389c/go.mod h1:+bmmJDNmKlhWNG+gwWCkaBoTy39Fs+bzRxVBzoTQbIc= @@ -127,14 +123,12 @@ github.com/erikstmartin/go-testdb v0.0.0-20160219214506-8d10e4a1bae5/go.mod h1:a github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/fatih/color v1.9.0 h1:8xPHl4/q1VyqGIPif1F+1V3Y3lSmrq01EabUW3CoW5s= github.com/fatih/color v1.9.0/go.mod h1:eQcE1qtQxscV5RaZvpXrrb8Drkc3/DdQ+uUYCNjL+zU= -github.com/fatih/structtag v1.2.0 h1:/OdNE99OxoI/PqaW/SuSK9uxxT3f/tcSZgon/ssNSx4= github.com/fatih/structtag v1.2.0/go.mod h1:mBJUNpUnHmRKrKlQQlmCrh5PuhftFbNv8Ys4/aAZl94= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsouza/fake-gcs-server v1.17.0 h1:OeH75kBZcZa3ZE+zz/mFdJ2btt9FgqfjI7gIh9+5fvk= github.com/fsouza/fake-gcs-server v1.17.0/go.mod h1:D1rTE4YCyHFNa99oyJJ5HyclvN/0uQR+pM/VdlL83bw= -github.com/ghodss/yaml v1.0.0 h1:wQHKEahhL6wmXdzwWG11gIVCkOv05bNOh+Rxn0yngAk= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/gin-contrib/gzip v0.0.1/go.mod h1:fGBJBCdt6qCZuCAOwWuFhBB4OOq9EFqlo5dEaFhhu5w= github.com/gin-contrib/sse v0.0.0-20170109093832-22d885f9ecc7/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= @@ -155,19 +149,15 @@ github.com/go-ole/go-ole v1.2.4 h1:nNBDSCOigTSiarFpYE9J/KtEA1IOW4CNeqT9TQDqCxI= github.com/go-ole/go-ole v1.2.4/go.mod h1:XCwSNxSkXRo4vlyPy93sltvi/qJq0jqQhjqQNIwKuxM= github.com/go-openapi/jsonpointer v0.17.0/go.mod h1:cOnomiV+CVVwFLk0A/MExoFMjwdsUdVpsRhURCKh+3M= github.com/go-openapi/jsonpointer v0.19.2/go.mod h1:3akKfEdA7DF1sugOqz1dVQHBcuDBPKZGEoHC/NkiQRg= -github.com/go-openapi/jsonpointer v0.19.3 h1:gihV7YNZK1iK6Tgwwsxo2rJbD1GTbdm72325Bq8FI3w= github.com/go-openapi/jsonpointer v0.19.3/go.mod h1:Pl9vOtqEWErmShwVjC8pYs9cog34VGT37dQOVbmoatg= github.com/go-openapi/jsonreference v0.17.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.0/go.mod h1:g4xxGn04lDIRh0GJb5QlpE3HfopLOL6uZrK/VgnsK9I= github.com/go-openapi/jsonreference v0.19.2/go.mod h1:jMjeRr2HHw6nAVajTXJ4eiUwohSTlpa0o73RUL1owJc= -github.com/go-openapi/jsonreference v0.19.3 h1:5cxNfTy0UVC3X8JL5ymxzyoUZmo8iZb+jeTWn7tUa8o= github.com/go-openapi/jsonreference v0.19.3/go.mod h1:rjx6GuL8TTa9VaixXglHmQmIL98+wF9xc8zWvFonSJ8= github.com/go-openapi/spec v0.19.0/go.mod h1:XkF/MOi14NmjsfZ8VtAKf8pIlbZzyoTvZsdfssdxcBI= -github.com/go-openapi/spec v0.19.4 h1:ixzUSnHTd6hCemgtAJgluaTSGYpLNpJY4mA2DIkdOAo= github.com/go-openapi/spec v0.19.4/go.mod h1:FpwSN1ksY1eteniUU7X0N/BgJ7a4WvBFVA8Lj9mJglo= github.com/go-openapi/swag v0.17.0/go.mod h1:AByQ+nYG6gQg71GINrmuDXCPWdL640yX49/kXLo40Tg= github.com/go-openapi/swag v0.19.2/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= -github.com/go-openapi/swag v0.19.5 h1:lTz6Ys4CmqqCQmZPBlbQENR1/GucA2bzYTE12Pw4tFY= github.com/go-openapi/swag v0.19.5/go.mod h1:POnQmlKehdgb5mhVOsnJFsivZCEZ/vjK9gh66Z9tfKk= github.com/go-playground/locales v0.12.1/go.mod h1:IUMDtCfWo/w/mtMfIE/IG2K+Ey3ygWanZIBtBW0W2TM= github.com/go-playground/overalls v0.0.0-20180201144345-22ec1a223b7c/go.mod h1:UqxAgEOt89sCiXlrc/ycnx00LVvUO/eS8tMUkWX4R7w= @@ -301,7 +291,6 @@ github.com/lib/pq v1.1.1/go.mod h1:5WUZQaWbwv1U+lTReE5YruASi9Al49XbQIvNi/34Woo= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190614124828-94de47d64c63/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= -github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e h1:hB2xlXdHp/pmPZq0y3QnmWAArdw9PqbmotexnWx/FU8= github.com/mailru/easyjson v0.0.0-20190626092158-b2ccc519800e/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= @@ -317,7 +306,6 @@ github.com/mattn/go-isatty v0.0.11/go.mod h1:PhnuNfih5lzO57/f3n+odYbM4JtupLOxQOA github.com/mattn/go-isatty v0.0.12 h1:wuysRhFDzyxgEmMf5xjvJ2M9dZoWAXNNr5LSBS7uHXY= github.com/mattn/go-isatty v0.0.12/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= -github.com/mattn/go-runewidth v0.0.7 h1:Ei8KR0497xHyKJPAv59M1dkC+rOZCMBJ+t3fZ+twI54= github.com/mattn/go-runewidth v0.0.7/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= github.com/mattn/go-runewidth v0.0.9 h1:Lm995f3rfxdpd6TSmuVCHVb/QhupuXlYr8sCI/QdE+0= github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= @@ -325,11 +313,8 @@ github.com/mattn/go-shellwords v1.0.3/go.mod h1:3xCvwCdWdlDJUrvuMn7Wuy9eWs4pE8vq github.com/mattn/go-sqlite3 v2.0.1+incompatible/go.mod h1:FPy6KqzDD04eiIsT53CuJW3U88zkxoIYsOqkbpncsNc= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= -github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81 h1:QASJXOGm2RZ5Ardbc86qNFvby9AqkLDibfChMtAg5QM= github.com/mgechev/dots v0.0.0-20190921121421-c36f7dcfbb81/go.mod h1:KQ7+USdGKfpPjXk4Ga+5XxQM4Lm4e3gAogrreFAYpOg= -github.com/mgechev/revive v1.0.2 h1:v0NxxQ7fSFz/u1NQydPo6EGdq7va0J1BtsZmae6kzUg= github.com/mgechev/revive v1.0.2/go.mod h1:rb0dQy1LVAxW9SWy5R3LPUjevzUbUS316U5MFySA2lo= -github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= github.com/modern-go/concurrent v0.0.0-20180228061459-e0a39a4cb421/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= @@ -350,7 +335,6 @@ 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/oleiade/reflections v1.0.0/go.mod h1:RbATFBbKYkVdqmSFtx13Bb/tVhR0lgOBXunWTZKeL4w= github.com/olekukonko/tablewriter v0.0.0-20170122224234-a0225b3f23b5/go.mod h1:vsDQFd/mU46D+Z4whnwzcISnGGzXWMclvtLoiIKAKIo= -github.com/olekukonko/tablewriter v0.0.4 h1:vHD/YYe1Wolo78koG299f7V/VAS08c6IpCLn+Ejf/w8= github.com/olekukonko/tablewriter v0.0.4/go.mod h1:zq6QwlOf5SlnkVbMSr5EoBv3636FWnp+qbPhuoO21uA= 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= @@ -365,7 +349,6 @@ github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsq github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= github.com/pelletier/go-toml v1.3.0/go.mod h1:PN7xzY2wHTK0K9p34ErDQMlFxa51Fk0OUruD3k1mMwo= -github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5 h1:q2e307iGHPdTGp0hoxKjt1H5pDo6utceo3dQVK3I5XQ= github.com/petermattis/goid v0.0.0-20180202154549-b0b1615b78e5/go.mod h1:jvVRKCrJTQWu0XVbaOlby/2lO20uSCHEMzzplHXte1o= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5etusZG3Cf+rpow5hqQByeCzJ2g= github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= @@ -381,7 +364,6 @@ github.com/pingcap/errcode v0.3.0/go.mod h1:4b2X8xSqxIroj/IZ9MX/VGZhAwc11wB9wRIz github.com/pingcap/errors v0.11.0/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= github.com/pingcap/errors v0.11.5-0.20190809092503-95897b64e011/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= -github.com/pingcap/errors v0.11.5-0.20201029093017-5a7df2af2ac7 h1:wQKuKP2HUtej2gSvx1cZmY4DENUH6tlOxRkfvPT8EBU= github.com/pingcap/errors v0.11.5-0.20201029093017-5a7df2af2ac7/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 h1:LllgC9eGfqzkfubMgjKIDyZYaa609nNWAyNZtpy2B3M= github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3/go.mod h1:G7x87le1poQzLB/TqvTJI2ILrSgobnq4Ut7luOwvfvI= @@ -394,7 +376,6 @@ github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989 h1:surzm05a8C9dN github.com/pingcap/goleveldb v0.0.0-20191226122134-f82aafb29989/go.mod h1:O17XtbryoCJhkKGbT62+L2OlrniwqiGLSqrmdHCMzZw= github.com/pingcap/kvproto v0.0.0-20191211054548-3c6b38ea5107/go.mod h1:WWLmULLO7l8IOcQG+t+ItJ3fEcrL5FxF0Wu+HrMy26w= github.com/pingcap/kvproto v0.0.0-20200411081810-b85805c9476c/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= -github.com/pingcap/kvproto v0.0.0-20200907074027-32a3a0accf7d h1:gvJScINTd/HFasp82W1paGTfbYe2Bnzn8QBOJXckLwY= github.com/pingcap/kvproto v0.0.0-20200907074027-32a3a0accf7d/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= github.com/pingcap/kvproto v0.0.0-20210308075244-560097d1309b h1:Jp0V5PDzdOy666n4XbDDaEjOKHsp2nk7b2uR6qjFI0s= github.com/pingcap/kvproto v0.0.0-20210308075244-560097d1309b/go.mod h1:IOdRDPLyda8GX2hE/jO7gqaCV/PNFh8BZQCQZXfIOqI= @@ -451,7 +432,6 @@ github.com/rs/cors v1.7.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= github.com/sasha-s/go-deadlock v0.2.0/go.mod h1:StQn567HiB1fF2yJ44N9au7wOhrPS3iZqiDbRupzT10= github.com/satori/go.uuid v1.2.0/go.mod h1:dA0hQrYB0VpLJoorglMZABFdXlWrHn1NEOzdhQKdks0= -github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44 h1:tB9NOR21++IjLyVx3/PCPhWMwqGNCMQEH96A6dMZ/gc= github.com/sergi/go-diff v1.0.1-0.20180205163309-da645544ed44/go.mod h1:0CfEIISq7TuYL3j771MWULgwwjU+GofnZX9QAmXWZgo= github.com/shirou/gopsutil v2.19.10+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= github.com/shirou/gopsutil v2.20.3+incompatible h1:0JVooMPsT7A7HqEYdydp/OfjSOYSjhXV7w1hkKj/NPQ= diff --git a/infoschema/builder.go b/infoschema/builder.go index 964ccf0737589..0cb835368efd8 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -119,20 +119,16 @@ func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoi switch diff.Type { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: // Only drop auto-increment allocator. - for _, alloc := range oldAllocs { - if alloc.GetType() == autoid.RowIDAllocType || alloc.GetType() == autoid.AutoIncrementType { - continue - } - newAllocs = append(newAllocs, alloc) - } + newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool { + tp := a.GetType() + return tp != autoid.RowIDAllocType && tp != autoid.AutoIncrementType + }) case model.ActionRebaseAutoRandomBase: // Only drop auto-random allocator. - for _, alloc := range oldAllocs { - if alloc.GetType() == autoid.AutoRandomType { - continue - } - newAllocs = append(newAllocs, alloc) - } + newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool { + tp := a.GetType() + return tp != autoid.AutoRandomType + }) default: // Keep all allocators. newAllocs = oldAllocs @@ -264,6 +260,16 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i case model.ActionRebaseAutoRandomBase: newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) allocs = append(allocs, newAlloc) + case model.ActionModifyColumn: + // Change column attribute from auto_increment to auto_random. + if tblInfo.ContainsAutoRandomBits() && allocs.Get(autoid.AutoRandomType) == nil { + // Remove auto_increment allocator. + allocs = allocs.Filter(func(a autoid.Allocator) bool { + return a.GetType() != autoid.AutoIncrementType && a.GetType() != autoid.RowIDAllocType + }) + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + allocs = append(allocs, newAlloc) + } } } tbl, err := tables.TableFromMeta(allocs, tblInfo) diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 318c297b0f26a..7b84226461925 100755 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -68,7 +68,7 @@ const MaxAutoRandomBits = 15 var step = int64(30000) // AllocatorType is the type of allocator for generating auto-id. Different type of allocators use different key-value pairs. -type AllocatorType = uint8 +type AllocatorType uint8 const ( // RowIDAllocType indicates the allocator is used to allocate row id. @@ -81,11 +81,28 @@ const ( SequenceType ) +func (a AllocatorType) String() string { + switch a { + case RowIDAllocType: + return "_tidb_rowid" + case AutoIncrementType: + return "auto_increment" + case AutoRandomType: + return "auto_random" + case SequenceType: + return "sequence" + } + return "unknown" +} + // CustomAutoIncCacheOption is one kind of AllocOption to customize the allocator step length. type CustomAutoIncCacheOption int64 // ApplyOn is implement the AllocOption interface. func (step CustomAutoIncCacheOption) ApplyOn(alloc *allocator) { + if step == 0 { + return + } alloc.step = int64(step) alloc.customStep = true } @@ -147,6 +164,17 @@ func (all Allocators) Get(allocType AllocatorType) Allocator { return nil } +// Filter filters all the allocators that match pred. +func (all Allocators) Filter(pred func(Allocator) bool) Allocators { + var ret Allocators + for _, a := range all { + if pred(a) { + ret = append(ret, a) + } + } + return ret +} + type allocator struct { mu sync.Mutex base int64 @@ -189,7 +217,7 @@ func (alloc *allocator) NextGlobalAutoID(tableID int64) (int64, error) { err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { var err1 error m := meta.NewMeta(txn) - autoID, err1 = getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + autoID, err1 = GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err1 != nil { return errors.Trace(err1) } @@ -216,7 +244,7 @@ func (alloc *allocator) rebase4Unsigned(tableID int64, requiredBase uint64, allo startTime := time.Now() err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) - currentEnd, err1 := getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + currentEnd, err1 := GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err1 != nil { return err1 } @@ -237,7 +265,7 @@ func (alloc *allocator) rebase4Unsigned(tableID int64, requiredBase uint64, allo newBase = requiredBase newEnd = requiredBase } - _, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, int64(newEnd-uCurrentEnd), alloc.allocType) + _, err1 = GenerateAutoID(m, alloc.dbID, tableID, int64(newEnd-uCurrentEnd), alloc.allocType) return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDRebase, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -262,7 +290,7 @@ func (alloc *allocator) rebase4Signed(tableID, requiredBase int64, allocIDs bool startTime := time.Now() err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) - currentEnd, err1 := getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + currentEnd, err1 := GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err1 != nil { return err1 } @@ -282,7 +310,7 @@ func (alloc *allocator) rebase4Signed(tableID, requiredBase int64, allocIDs bool newBase = requiredBase newEnd = requiredBase } - _, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, newEnd-currentEnd, alloc.allocType) + _, err1 = GenerateAutoID(m, alloc.dbID, tableID, newEnd-currentEnd, alloc.allocType) return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDRebase, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -299,7 +327,7 @@ func (alloc *allocator) rebase4Sequence(tableID, requiredBase int64) (int64, boo alreadySatisfied := false err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) - currentEnd, err := getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + currentEnd, err := GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err != nil { return err } @@ -320,7 +348,7 @@ func (alloc *allocator) rebase4Sequence(tableID, requiredBase int64) (int64, boo // If we don't want to allocate IDs, for example when creating a table with a given base value, // We need to make sure when other TiDB server allocates ID for the first time, requiredBase + 1 // will be allocated, so we need to increase the end to exactly the requiredBase. - _, err = generateAutoIDByAllocType(m, alloc.dbID, tableID, requiredBase-currentEnd, alloc.allocType) + _, err = GenerateAutoID(m, alloc.dbID, tableID, requiredBase-currentEnd, alloc.allocType) return err }) // TODO: sequence metrics @@ -427,13 +455,18 @@ func NewSequenceAllocator(store kv.Storage, dbID int64, info *model.SequenceInfo func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.TableInfo) Allocators { var allocs []Allocator dbID := tblInfo.GetDBID(schemaID) - if tblInfo.AutoIdCache > 0 { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, CustomAutoIncCacheOption(tblInfo.AutoIdCache))) - } else { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType)) + idCacheOpt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) + + hasRowID := !tblInfo.PKIsHandle + hasAutoIncID := tblInfo.GetAutoIncrementColInfo() != nil + if hasRowID || hasAutoIncID { + alloc := NewAllocator(store, dbID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, idCacheOpt) + allocs = append(allocs, alloc) } - if tblInfo.ContainsAutoRandomBits() { - allocs = append(allocs, NewAllocator(store, dbID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType)) + hasAutoRandID := tblInfo.ContainsAutoRandomBits() + if hasAutoRandID { + alloc := NewAllocator(store, dbID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType, idCacheOpt) + allocs = append(allocs, alloc) } if tblInfo.IsSequence() { allocs = append(allocs, NewSequenceAllocator(store, dbID, tblInfo.Sequence)) @@ -648,7 +681,7 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error - newBase, err1 = getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + newBase, err1 = GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err1 != nil { return err1 } @@ -663,7 +696,7 @@ func (alloc *allocator) alloc4Signed(tableID int64, n uint64, increment, offset if tmpStep < n1 { return ErrAutoincReadFailed } - newEnd, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, tmpStep, alloc.allocType) + newEnd, err1 = GenerateAutoID(m, alloc.dbID, tableID, tmpStep, alloc.allocType) return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -717,7 +750,7 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error { m := meta.NewMeta(txn) var err1 error - newBase, err1 = getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + newBase, err1 = GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err1 != nil { return err1 } @@ -732,7 +765,7 @@ func (alloc *allocator) alloc4Unsigned(tableID int64, n uint64, increment, offse if tmpStep < n1 { return ErrAutoincReadFailed } - newEnd, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, tmpStep, alloc.allocType) + newEnd, err1 = GenerateAutoID(m, alloc.dbID, tableID, tmpStep, alloc.allocType) return err1 }) metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) @@ -802,7 +835,7 @@ func (alloc *allocator) alloc4Sequence(tableID int64) (min int64, max int64, rou } // Get the global new base. - newBase, err1 = getAutoIDByAllocType(m, alloc.dbID, tableID, alloc.allocType) + newBase, err1 = GetAutoID(m, alloc.dbID, tableID, alloc.allocType) if err1 != nil { return err1 } @@ -849,7 +882,7 @@ func (alloc *allocator) alloc4Sequence(tableID int64) (min int64, max int64, rou } else { delta = -seqStep } - newEnd, err1 = generateAutoIDByAllocType(m, alloc.dbID, tableID, delta, alloc.allocType) + newEnd, err1 = GenerateAutoID(m, alloc.dbID, tableID, delta, alloc.allocType) return err1 }) @@ -866,7 +899,8 @@ func (alloc *allocator) alloc4Sequence(tableID int64) (min int64, max int64, rou return newBase, newEnd, round, nil } -func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType AllocatorType) (int64, error) { +// GetAutoID get an auto ID for the given allocator type. +func GetAutoID(m *meta.Meta, dbID, tableID int64, allocType AllocatorType) (int64, error) { switch allocType { // Currently, row id allocator and auto-increment value allocator shares the same key-value pair. case RowIDAllocType, AutoIncrementType: @@ -880,7 +914,8 @@ func getAutoIDByAllocType(m *meta.Meta, dbID, tableID int64, allocType Allocator } } -func generateAutoIDByAllocType(m *meta.Meta, dbID, tableID, step int64, allocType AllocatorType) (int64, error) { +// GenerateAutoID generate an auto ID for the given allocator type. +func GenerateAutoID(m *meta.Meta, dbID, tableID, step int64, allocType AllocatorType) (int64, error) { switch allocType { case RowIDAllocType, AutoIncrementType: return m.GenAutoTableID(dbID, tableID, step) diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go index c8152c6b810ba..10ab2499ce3e7 100644 --- a/meta/autoid/errors.go +++ b/meta/autoid/errors.go @@ -55,4 +55,10 @@ const ( AutoRandomRebaseNotApplicable = "alter auto_random_base of a non auto_random table" // AutoRandomRebaseOverflow is reported when alter auto_random_base to a value that overflows the incremental bits. AutoRandomRebaseOverflow = "alter auto_random_base to %d overflows the incremental bits, max allowed base is %d" + // AutoRandomAlterAddColumn is reported when adding an auto_random column. + AutoRandomAlterAddColumn = "unsupported add column '%s' constraint AUTO_RANDOM when altering '%s.%s'" + // AutoRandomAlterChangeFromAutoInc is reported when the column is changing from a non-auto_increment or a non-primary key. + AutoRandomAlterChangeFromAutoInc = "auto_random can only be converted from auto_increment clustered primary key" + // AutoRandomAllocatorNotFound is reported when auto_random ID allocator not found during changing from auto_inc to auto_random. + AutoRandomAllocatorNotFound = "auto_random ID allocator not found in table '%s.%s'" ) diff --git a/meta/meta.go b/meta/meta.go index 3c077486254d2..a7e69786fa799 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -434,6 +434,15 @@ func (m *Meta) DropTableOrView(dbID int64, tblID int64, delAutoID bool) error { return nil } +// CleanAutoID is used to delete the auto-id of specific table. +func (m *Meta) CleanAutoID(dbID, tblID int64) error { + dbKey := m.dbKey(dbID) + if err := m.txn.HDel(dbKey, m.autoTableIDKey(tblID)); err != nil { + return errors.Trace(err) + } + return nil +} + // UpdateTable updates the table with table info. func (m *Meta) UpdateTable(dbID int64, tableInfo *model.TableInfo) error { // Check if db exists.