Skip to content

Commit

Permalink
*: support auto_random table option (pingcap#16750) (pingcap#16812)
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored Apr 26, 2020
1 parent 63dd2e1 commit 489b077
Show file tree
Hide file tree
Showing 16 changed files with 302 additions and 28 deletions.
37 changes: 29 additions & 8 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1564,7 +1564,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)
}
}

Expand Down Expand Up @@ -1847,7 +1854,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)
Expand All @@ -1857,8 +1864,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
}
Expand All @@ -1884,6 +1897,8 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err
switch op.Tp {
case ast.TableOptionAutoIncrement:
tbInfo.AutoIncID = int64(op.UintValue)
case ast.TableOptionAutoRandomBase:
tbInfo.AutoRandID = int64(op.UintValue)
case ast.TableOptionComment:
tbInfo.Comment = op.StrValue
case ast.TableOptionCompression:
Expand Down Expand Up @@ -2067,7 +2082,9 @@ 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.TableOptionAutoRandomBase:
err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.AutoRandomType)
case ast.TableOptionComment:
spec.Comment = opt.StrValue
err = d.AlterTableComment(ctx, ident, spec)
Expand Down Expand Up @@ -2104,12 +2121,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.Allocator(ctx, autoid.RowIDAllocType).NextGlobalAutoID(t.Meta().ID)
autoIncID, err := t.Allocator(ctx, tp).NextGlobalAutoID(t.Meta().ID)
if err != nil {
return errors.Trace(err)
}
Expand All @@ -2119,11 +2136,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},
}
Expand Down
4 changes: 3 additions & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -604,7 +604,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:
Expand Down
19 changes: 16 additions & 3 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -460,7 +460,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)
Expand All @@ -474,7 +482,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
Expand All @@ -483,7 +496,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)
}
Expand Down
127 changes: 126 additions & 1 deletion executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -821,7 +821,7 @@ func (s *testSuite3) 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)")

Expand Down Expand Up @@ -909,6 +909,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 *testSuite3) TestMaxHandleAddIndex(c *C) {
tk := testkit.NewTestKit(c, s.store)

Expand Down
4 changes: 2 additions & 2 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -575,7 +575,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
}
Expand Down Expand Up @@ -662,7 +662,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
}
Expand Down
4 changes: 4 additions & 0 deletions executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -878,6 +878,10 @@ func ConstructResultOfShowCreateTable(ctx sessionctx.Context, tableInfo *model.T
}
}

if tableInfo.AutoRandID != 0 {
fmt.Fprintf(buf, " /*T![auto_rand_base] 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 {
Expand Down
29 changes: 29 additions & 0 deletions executor/show_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -699,4 +699,33 @@ 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_base] 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_base] 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_base] AUTO_RANDOM_BASE=200 */",
))
}
2 changes: 1 addition & 1 deletion executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,7 +94,7 @@ func updateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datu
if err != nil {
return false, false, 0, err
}
if err = t.RebaseAutoID(ctx, recordID, true); err != nil {
if err = t.RebaseAutoID(ctx, recordID, true, autoid.RowIDAllocType); err != nil {
return false, false, 0, err
}
}
Expand Down
Loading

0 comments on commit 489b077

Please sign in to comment.