diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 7df098d39990e..86f3398a763a3 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1579,7 +1579,14 @@ 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. - err = d.handleAutoIncID(tbInfo, schema.ID) + if err = d.handleAutoIncID(tbInfo, schema.ID, 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) } } @@ -1869,7 +1876,7 @@ 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) error { +func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, tp autoid.AllocatorType) error { allocs := autoid.NewAllocatorsFromTblInfo(d.store, schemaID, tbInfo) tbInfo.State = model.StatePublic tb, err := table.TableFromMeta(allocs, tbInfo) @@ -1879,8 +1886,14 @@ func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64) error { // 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 err = tb.RebaseAutoID(nil, tbInfo.AutoIncID-1, false); err != nil { - return errors.Trace(err) + 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 { + return errors.Trace(err) + } } return nil } @@ -1897,6 +1910,8 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err return errors.New("table option auto_id_cache overflows int64") } tbInfo.AutoIdCache = int64(op.UintValue) + case ast.TableOptionAutoRandomBase: + tbInfo.AutoRandID = int64(op.UintValue) case ast.TableOptionComment: tbInfo.Comment = op.StrValue case ast.TableOptionCompression: @@ -2128,13 +2143,15 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } err = d.ShardRowID(ctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue)) + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.RowIDAllocType) case ast.TableOptionAutoIdCache: if opt.UintValue > uint64(math.MaxInt64) { // TODO: Refine this error. return errors.New("table option auto_id_cache overflows int64") } err = d.AlterTableAutoIDCache(ctx, ident, int64(opt.UintValue)) + case ast.TableOptionAutoRandomBase: + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.AutoRandomType) case ast.TableOptionComment: spec.Comment = opt.StrValue err = d.AlterTableComment(ctx, ident, spec) @@ -2174,12 +2191,12 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return nil } -func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64) error { +func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp autoid.AllocatorType) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) if err != nil { return errors.Trace(err) } - autoIncID, err := t.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(t.Meta().ID) + autoIncID, err := t.Allocators(ctx).Get(tp).NextGlobalAutoID(t.Meta().ID) if err != nil { return errors.Trace(err) } @@ -2189,11 +2206,15 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 // 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 = mathutil.MaxInt64(newBase, autoIncID) + actionType := model.ActionRebaseAutoID + if tp == autoid.AutoRandomType { + actionType = model.ActionRebaseAutoRandomBase + } job := &model.Job{ SchemaID: schema.ID, TableID: t.Meta().ID, SchemaName: schema.Name.L, - Type: model.ActionRebaseAutoID, + Type: actionType, BinlogInfo: &model.HistoryInfo{}, Args: []interface{}{newBase}, } diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index c3d03c66e1243..a842877113e0a 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -624,7 +624,9 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionTruncateTable: ver, err = onTruncateTable(d, t, job) case model.ActionRebaseAutoID: - ver, err = onRebaseAutoID(d.store, t, job) + ver, err = onRebaseRowIDType(d.store, t, job) + case model.ActionRebaseAutoRandomBase: + ver, err = onRebaseAutoRandomType(d.store, t, job) case model.ActionRenameTable: ver, err = onRenameTable(d, t, job) case model.ActionShardRowID: diff --git a/ddl/table.go b/ddl/table.go index 170e5668d6a4b..921afdf9b34ac 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -481,7 +481,15 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, nil } -func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { +func onRebaseRowIDType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(store, t, job, autoid.RowIDAllocType) +} + +func onRebaseAutoRandomType(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(store, t, job, autoid.AutoRandomType) +} + +func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.AllocatorType) (ver int64, _ error) { schemaID := job.SchemaID var newBase int64 err := job.DecodeArgs(&newBase) @@ -495,7 +503,12 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, return ver, errors.Trace(err) } // No need to check `newBase` again, because `RebaseAutoID` will do this check. - tblInfo.AutoIncID = newBase + if tp == autoid.RowIDAllocType { + tblInfo.AutoIncID = newBase + } else { + tblInfo.AutoRandID = newBase + } + tbl, err := getTable(store, schemaID, tblInfo) if err != nil { job.State = model.JobStateCancelled @@ -504,7 +517,7 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64, // 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, tblInfo.AutoIncID-1, false) + err = tbl.RebaseAutoID(nil, newBase-1, false, tp) if err != nil { return ver, errors.Trace(err) } diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 394565f2715fd..e027548da0eec 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -780,7 +780,7 @@ func (s *testSuite8) TestShardRowIDBits(c *C) { tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) c.Assert(err, IsNil) maxID := 1<<(64-15-1) - 1 - err = tbl.RebaseAutoID(tk.Se, int64(maxID)-1, false) + err = tbl.RebaseAutoID(tk.Se, int64(maxID)-1, false, autoid.RowIDAllocType) c.Assert(err, IsNil) tk.MustExec("insert into t1 values(1)") @@ -868,6 +868,131 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) { tk.MustExec("drop table t") } +func (s *testAutoRandomSuite) TestAutoRandomTableOption(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + + // test table option is auto-random + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() + + tk.MustExec("drop table if exists auto_random_table_option") + tk.MustExec("create table auto_random_table_option (a bigint auto_random(5) key) auto_random_base = 1000") + t, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto_random_table_option")) + c.Assert(err, IsNil) + c.Assert(t.Meta().AutoRandID, Equals, int64(1000)) + tk.MustExec("insert into auto_random_table_option values (),(),(),(),()") + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test", "auto_random_table_option") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 5) + // Test the high bits of handles are not all zero. + allZero := true + for _, h := range allHandles { + allZero = allZero && (h>>(64-6)) == 0 + } + c.Assert(allZero, IsFalse) + // Test non-shard-bits part of auto random id is monotonic increasing and continuous. + orderedHandles := testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size := int64(len(allHandles)) + for i := int64(0); i < size; i++ { + c.Assert(i+1000, Equals, orderedHandles[i]) + } + + tk.MustExec("drop table if exists alter_table_auto_random_option") + tk.MustExec("create table alter_table_auto_random_option (a bigint primary key auto_random(4), b int)") + t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + c.Assert(err, IsNil) + c.Assert(t.Meta().AutoRandID, Equals, int64(0)) + tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "alter_table_auto_random_option") + c.Assert(err, IsNil) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < size; i++ { + c.Assert(orderedHandles[i], Equals, i+1) + } + tk.MustExec("delete from alter_table_auto_random_option") + + // alter table to change the auto_random option (it will dismiss the local allocator cache) + // To avoid the new base is in the range of local cache, which will leading the next + // value is not what we rebased, because the local cache is dropped, here we choose + // a quite big value to do this. + tk.MustExec("alter table alter_table_auto_random_option auto_random_base = 3000000") + t, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + c.Assert(err, IsNil) + c.Assert(t.Meta().AutoRandID, Equals, int64(3000000)) + tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "alter_table_auto_random_option") + c.Assert(err, IsNil) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < size; i++ { + c.Assert(orderedHandles[i], Equals, i+3000000) + } + tk.MustExec("drop table alter_table_auto_random_option") +} + +// Test filter different kind of allocators. +// In special ddl type, for example: +// 1: ActionRenameTable : it will abandon all the old allocators. +// 2: ActionRebaseAutoID : it will drop row-id-type allocator. +// 3: ActionModifyTableAutoIdCache : it will drop row-id-type allocator. +// 3: ActionRebaseAutoRandomBase : it will drop auto-rand-type allocator. +func (s *testAutoRandomSuite) TestFilterDifferentAllocators(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t") + tk.MustExec("drop table if exists t1") + + testutil.ConfigTestUtils.SetupAutoRandomTestConfig() + defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig() + + tk.MustExec("create table t(a bigint auto_random(5) key, b int auto_increment unique)") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("1")) + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles := testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Equals, int64(1)) + tk.MustExec("delete from t") + + // Test rebase auto_increment. + tk.MustExec("alter table t auto_increment 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000000")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Equals, int64(2)) + tk.MustExec("delete from t") + + // Test rebase auto_random. + tk.MustExec("alter table t auto_random_base 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000001")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Equals, int64(3000000)) + tk.MustExec("delete from t") + + // Test rename table. + tk.MustExec("rename table t to t1") + tk.MustExec("insert into t1 values()") + res := tk.MustQuery("select b from t1") + strInt64, err := strconv.ParseInt(res.Rows()[0][0].(string), 10, 64) + c.Assert(err, IsNil) + c.Assert(strInt64, Greater, int64(3000002)) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Se, "test", "t1") + c.Assert(err, IsNil) + c.Assert(len(allHandles), Equals, 1) + orderedHandles = testutil.ConfigTestUtils.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + c.Assert(orderedHandles[0], Greater, int64(3000001)) +} + func (s *testSuite6) TestMaxHandleAddIndex(c *C) { tk := testkit.NewTestKit(c, s.store) diff --git a/executor/insert_common.go b/executor/insert_common.go index aa0116e2f6c4d..10993942abc68 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -682,7 +682,7 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows [] } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.RebaseAutoID(e.ctx, recordID, true) + err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.RowIDAllocType) if err != nil { return nil, err } @@ -769,7 +769,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.RebaseAutoID(e.ctx, recordID, true) + err = e.Table.RebaseAutoID(e.ctx, recordID, true, autoid.RowIDAllocType) if err != nil { return types.Datum{}, err } diff --git a/executor/show.go b/executor/show.go index 3005ebd3671b2..5c737f3ae5d99 100644 --- a/executor/show.go +++ b/executor/show.go @@ -911,6 +911,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T fmt.Fprintf(buf, " /*T![auto_id_cache] AUTO_ID_CACHE=%d */", tableInfo.AutoIdCache) } + if tableInfo.AutoRandID != 0 { + fmt.Fprintf(buf, " /*T![auto_rand] AUTO_RANDOM_BASE=%d */", tableInfo.AutoRandID) + } + if tableInfo.ShardRowIDBits > 0 { fmt.Fprintf(buf, "/*!90000 SHARD_ROW_ID_BITS=%d ", tableInfo.ShardRowIDBits) if tableInfo.PreSplitRegions > 0 { diff --git a/executor/show_test.go b/executor/show_test.go index 0abe79eb80f59..8ffd4d4a42dd6 100644 --- a/executor/show_test.go +++ b/executor/show_test.go @@ -716,6 +716,35 @@ func (s *testAutoRandomSuite) TestShowCreateTableAutoRandom(c *C) { " PRIMARY KEY (`a`)\n"+ ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin", )) + // Test show auto_random table option. + tk.MustExec("create table auto_random_tbl4 (a bigint primary key auto_random(5), b varchar(255)) auto_random_base = 100") + tk.MustQuery("show create table `auto_random_tbl4`").Check(testutil.RowsWithSep("|", + ""+ + "auto_random_tbl4 CREATE TABLE `auto_random_tbl4` (\n"+ + " `a` bigint(20) NOT NULL /*T![auto_rand] AUTO_RANDOM(5) */,\n"+ + " `b` varchar(255) DEFAULT NULL,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_rand] AUTO_RANDOM_BASE=100 */", + )) + // Test implicit auto_random with auto_random table option. + tk.MustExec("create table auto_random_tbl5 (a bigint auto_random primary key, b char) auto_random_base 50") + tk.MustQuery("show create table auto_random_tbl5").Check(testutil.RowsWithSep("|", + ""+ + "auto_random_tbl5 CREATE TABLE `auto_random_tbl5` (\n"+ + " `a` bigint(20) NOT NULL /*T![auto_rand] AUTO_RANDOM(5) */,\n"+ + " `b` char(1) DEFAULT NULL,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_rand] AUTO_RANDOM_BASE=50 */", + )) + // Test auto_random table option already with special comment. + tk.MustExec("create table auto_random_tbl6 (a bigint /*T![auto_rand] auto_random */ primary key) auto_random_base 200") + tk.MustQuery("show create table auto_random_tbl6").Check(testutil.RowsWithSep("|", + ""+ + "auto_random_tbl6 CREATE TABLE `auto_random_tbl6` (\n"+ + " `a` bigint(20) NOT NULL /*T![auto_rand] AUTO_RANDOM(5) */,\n"+ + " PRIMARY KEY (`a`)\n"+ + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_rand] AUTO_RANDOM_BASE=200 */", + )) } // Override testAutoRandomSuite to test auto id cache. diff --git a/executor/write.go b/executor/write.go index f82ca040c50d3..6e9c1a5052244 100644 --- a/executor/write.go +++ b/executor/write.go @@ -107,7 +107,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h int64, oldData if err != nil { return false, false, 0, err } - if err = t.RebaseAutoID(sctx, recordID, true); err != nil { + if err = t.RebaseAutoID(sctx, recordID, true, autoid.RowIDAllocType); err != nil { return false, false, 0, err } } diff --git a/infoschema/builder.go b/infoschema/builder.go index 4f4bfd34f33e0..c58c8499c907f 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -74,8 +74,9 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro // We try to reuse the old allocator, so the cached auto ID can be reused. var allocs autoid.Allocators if tableIDIsValid(oldTableID) { - if oldTableID == newTableID && diff.Type != model.ActionRenameTable && diff.Type != model.ActionRebaseAutoID && diff.Type != model.ActionModifyTableAutoIdCache { - allocs, _ = b.is.AllocByID(oldTableID) + if oldTableID == newTableID && diff.Type != model.ActionRenameTable { + oldAllocs, _ := b.is.AllocByID(oldTableID) + allocs = filterAllocators(diff, oldAllocs) } tmpIDs := tblIDs @@ -108,6 +109,32 @@ func (b *Builder) ApplyDiff(m *meta.Meta, diff *model.SchemaDiff) ([]int64, erro return tblIDs, nil } +func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators { + var newAllocs autoid.Allocators + 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) + } + case model.ActionRebaseAutoRandomBase: + // Only drop auto-random allocator. + for _, alloc := range oldAllocs { + if alloc.GetType() == autoid.AutoRandomType { + continue + } + newAllocs = append(newAllocs, alloc) + } + default: + // Keep all allocators. + newAllocs = oldAllocs + } + return newAllocs +} + func appendAffectedIDs(affected []int64, tblInfo *model.TableInfo) []int64 { affected = append(affected, tblInfo.ID) if pi := tblInfo.GetPartitionInfo(); pi != nil { @@ -224,6 +251,15 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i if len(allocs) == 0 { allocs = autoid.NewAllocatorsFromTblInfo(b.handle.store, dbInfo.ID, tblInfo) + } else { + switch tp { + case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType) + allocs = append(allocs, newAlloc) + case model.ActionRebaseAutoRandomBase: + newAlloc := autoid.NewAllocator(b.handle.store, dbInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType) + allocs = append(allocs, newAlloc) + } } tbl, err := tables.TableFromMeta(allocs, tblInfo) if err != nil { diff --git a/infoschema/tables.go b/infoschema/tables.go index 9f2bf44d2564a..4ea200576f3cb 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1654,7 +1654,7 @@ func (it *infoschemaTable) Allocators(_ sessionctx.Context) autoid.Allocators { } // RebaseAutoID implements table.Table RebaseAutoID interface. -func (it *infoschemaTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { +func (it *infoschemaTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { return table.ErrUnsupportedOp } @@ -1781,7 +1781,7 @@ func (vt *VirtualTable) Allocators(_ sessionctx.Context) autoid.Allocators { } // RebaseAutoID implements table.Table RebaseAutoID interface. -func (vt *VirtualTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { +func (vt *VirtualTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { return table.ErrUnsupportedOp } diff --git a/sessionctx/binloginfo/binloginfo.go b/sessionctx/binloginfo/binloginfo.go index 608abe4ecdc3c..8f745a1a99570 100644 --- a/sessionctx/binloginfo/binloginfo.go +++ b/sessionctx/binloginfo/binloginfo.go @@ -314,6 +314,7 @@ func AddSpecialComment(ddlQuery string) string { } // addSpecialCommentByRegexps uses to add special comment for the worlds in the ddlQuery with match the regexps. +// addSpecialCommentByRegexps will merge multi pattern regs to one special comment. func addSpecialCommentByRegexps(ddlQuery string, prefix string, regs ...*regexp.Regexp) string { upperQuery := strings.ToUpper(ddlQuery) var specialComments []string diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 153f88f70901e..001009c378949 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -559,6 +559,14 @@ func (s *testBinlogSuite) TestAddSpecialComment(c *C) { "create table t1 (id int auto_random ( 4 ) primary key);", "create table t1 (id int /*T![auto_rand] auto_random ( 4 ) */ primary key);", }, + { + "create table t1 (id int auto_random ( 3 ) primary key) auto_random_base = 100;", + "create table t1 (id int /*T![auto_rand] auto_random ( 3 ) */ primary key) /*T![auto_rand_base] auto_random_base = 100 */ ;", + }, + { + "create table t1 (id int auto_random primary key) auto_random_base = 50;", + "create table t1 (id int /*T![auto_rand] auto_random */ primary key) /*T![auto_rand_base] auto_random_base = 50 */ ;", + }, { "create table t1 (id int auto_increment key) auto_id_cache 100;", "create table t1 (id int auto_increment key) /*T![auto_id_cache] auto_id_cache 100 */ ;", diff --git a/table/table.go b/table/table.go index 86e0eca8910ef..b94090b26bfd0 100644 --- a/table/table.go +++ b/table/table.go @@ -206,7 +206,7 @@ type Table interface { // RebaseAutoID rebases the auto_increment ID base. // If allocIDs is true, it will allocate some IDs and save to the cache. // If allocIDs is false, it will not allocate IDs. - RebaseAutoID(ctx sessionctx.Context, newBase int64, allocIDs bool) error + RebaseAutoID(ctx sessionctx.Context, newBase int64, allocIDs bool, tp autoid.AllocatorType) error // Meta returns TableInfo. Meta() *model.TableInfo diff --git a/table/tables/tables.go b/table/tables/tables.go index 3f1e9b85be41a..b462ddf3f75b2 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -1091,8 +1091,9 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { } // RebaseAutoID implements table.Table RebaseAutoID interface. -func (t *TableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { - return t.Allocators(ctx).Get(autoid.RowIDAllocType).Rebase(t.tableID, newBase, isSetStep) +// Both auto-increment and auto-random can use this function to do rebase on explicit newBase value (without shadow bits). +func (t *TableCommon) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool, tp autoid.AllocatorType) error { + return t.Allocators(ctx).Get(tp).Rebase(t.tableID, newBase, isSetStep) } // Seek implements table.Table Seek interface. diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index bb2bc25c0378c..df10d7ff0fb6e 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -161,7 +161,7 @@ func (ts *testSuite) TestBasic(c *C) { alc := tb.Allocators(nil).Get(autoid.RowIDAllocType) c.Assert(alc, NotNil) - err = tb.RebaseAutoID(nil, 0, false) + err = tb.RebaseAutoID(nil, 0, false, autoid.RowIDAllocType) c.Assert(err, IsNil) } @@ -388,7 +388,7 @@ func (ts *testSuite) TestTableFromMeta(c *C) { c.Assert(err, IsNil) maxID := 1<<(64-15-1) - 1 - err = tb.RebaseAutoID(tk.Se, int64(maxID), false) + err = tb.RebaseAutoID(tk.Se, int64(maxID), false, autoid.RowIDAllocType) c.Assert(err, IsNil) _, err = tables.AllocHandle(tk.Se, tb) diff --git a/types/parser_driver/special_cmt_ctrl.go b/types/parser_driver/special_cmt_ctrl.go index 10a1fb6756eb6..ad34ae2307985 100644 --- a/types/parser_driver/special_cmt_ctrl.go +++ b/types/parser_driver/special_cmt_ctrl.go @@ -33,6 +33,7 @@ import ( func init() { parser.SpecialCommentsController.Register(string(FeatureIDAutoRandom)) parser.SpecialCommentsController.Register(string(FeatureIDAutoIDCache)) + parser.SpecialCommentsController.Register(string(FeatureIDAutoRandomBase)) } // SpecialCommentVersionPrefix is the prefix of TiDB executable comments. @@ -55,10 +56,13 @@ const ( FeatureIDAutoRandom featureID = "auto_rand" // FeatureIDAutoIDCache is the `auto_id_cache` feature. FeatureIDAutoIDCache featureID = "auto_id_cache" + // FeatureIDAutoRandomBase is the `auto_random_base` feature. + FeatureIDAutoRandomBase featureID = "auto_rand_base" ) // FeatureIDPatterns is used to record special comments patterns. var FeatureIDPatterns = map[featureID]*regexp.Regexp{ - FeatureIDAutoRandom: regexp.MustCompile(`(?i)AUTO_RANDOM\s*(\(\s*\d+\s*\))?\s*`), - FeatureIDAutoIDCache: regexp.MustCompile(`(?i)AUTO_ID_CACHE\s*=?\s*\d+\s*`), + FeatureIDAutoRandom: regexp.MustCompile(`(?i)AUTO_RANDOM\s*(\(\s*\d+\s*\)|\s+)\s*`), + FeatureIDAutoIDCache: regexp.MustCompile(`(?i)AUTO_ID_CACHE\s*=?\s*\d+\s*`), + FeatureIDAutoRandomBase: regexp.MustCompile(`(?i)AUTO_RANDOM_BASE\s*=?\s*\d+\s*`), }