From fe00b958f1710f09c92c25ab3d4705a53e29b241 Mon Sep 17 00:00:00 2001 From: bb7133 Date: Wed, 12 Jun 2019 20:43:29 +0800 Subject: [PATCH] ddl, table: allow using SHARD_ROW_ID_BITS with auto_incremental columns (#10759) --- ddl/ddl.go | 2 +- ddl/ddl_api.go | 10 +-- executor/ddl_test.go | 105 ++++++++++++++++++----- executor/insert_common.go | 2 +- infoschema/tables.go | 76 +++++++++++----- sessionctx/binloginfo/binloginfo_test.go | 8 +- table/table.go | 7 +- table/tables/tables.go | 11 ++- table/tables/tables_test.go | 19 ++-- 9 files changed, 174 insertions(+), 66 deletions(-) diff --git a/ddl/ddl.go b/ddl/ddl.go index 957dadf8af88f..8f850f7c98ac2 100644 --- a/ddl/ddl.go +++ b/ddl/ddl.go @@ -98,7 +98,7 @@ var ( "unsupported drop integer primary key") errUnsupportedCharset = terror.ClassDDL.New(codeUnsupportedCharset, "unsupported charset %s collate %s") - errUnsupportedShardRowIDBits = terror.ClassDDL.New(codeUnsupportedShardRowIDBits, "unsupported shard_row_id_bits for table with auto_increment column.") + errUnsupportedShardRowIDBits = terror.ClassDDL.New(codeUnsupportedShardRowIDBits, "unsupported shard_row_id_bits for table with primary key as row id.") errBlobKeyWithoutLength = terror.ClassDDL.New(codeBlobKeyWithoutLength, "index for BLOB/TEXT column must specify a key length") errIncorrectPrefixKey = terror.ClassDDL.New(codeIncorrectPrefixKey, "Incorrect prefix key; the used key part isn't a string, the used length is longer than the key part, or the storage engine doesn't support unique prefix keys") errTooLongKey = terror.ClassDDL.New(codeTooLongKey, diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 2ba16ef6f3016..7d04adab570fa 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -1647,8 +1647,7 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err case ast.TableOptionCompression: tbInfo.Compression = op.StrValue case ast.TableOptionShardRowID: - ok, _ := hasAutoIncrementColumn(tbInfo) - if ok && op.UintValue != 0 { + if op.UintValue > 0 && tbInfo.PKIsHandle { return errUnsupportedShardRowIDBits } tbInfo.ShardRowIDBits = op.UintValue @@ -1905,14 +1904,13 @@ func (d *ddl) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint if err != nil { return errors.Trace(err) } - ok, _ := hasAutoIncrementColumn(t.Meta()) - if ok && uVal != 0 { - return errUnsupportedShardRowIDBits - } if uVal == t.Meta().ShardRowIDBits { // Nothing need to do. return nil } + if uVal > 0 && t.Meta().PKIsHandle { + return errUnsupportedShardRowIDBits + } err = verifyNoOverflowShardBits(d.sessPool, t, uVal) if err != nil { return err diff --git a/executor/ddl_test.go b/executor/ddl_test.go index ebfc667bbae42..d67ecbca3dfb3 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -17,6 +17,7 @@ import ( "context" "fmt" "math" + "strconv" "strings" "time" @@ -27,6 +28,8 @@ import ( "github.com/pingcap/tidb/ddl" ddlutil "github.com/pingcap/tidb/ddl/util" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx" @@ -522,38 +525,98 @@ func (s *testSuite3) TestShardRowIDBits(c *C) { for i := 0; i < 100; i++ { tk.MustExec(fmt.Sprintf("insert t values (%d)", i)) } - tbl, err := domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + dom := domain.GetDomain(tk.Se) + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) c.Assert(err, IsNil) - var hasShardedID bool - var count int - c.Assert(tk.Se.NewTxn(context.Background()), IsNil) - err = tbl.IterRecords(tk.Se, tbl.FirstKey(), nil, func(h int64, rec []types.Datum, cols []*table.Column) (more bool, err error) { - c.Assert(h, GreaterEqual, int64(0)) - first8bits := h >> 56 - if first8bits > 0 { - hasShardedID = true - } - count++ - return true, nil + + assertCountAndShard := func(t table.Table, expectCount int) { + var hasShardedID bool + var count int + c.Assert(tk.Se.NewTxn(context.Background()), IsNil) + err = t.IterRecords(tk.Se, t.FirstKey(), nil, func(h int64, rec []types.Datum, cols []*table.Column) (more bool, err error) { + c.Assert(h, GreaterEqual, int64(0)) + first8bits := h >> 56 + if first8bits > 0 { + hasShardedID = true + } + count++ + return true, nil + }) + c.Assert(err, IsNil) + c.Assert(count, Equals, expectCount) + c.Assert(hasShardedID, IsTrue) + } + + assertCountAndShard(tbl, 100) + + // After PR 10759, shard_row_id_bits is supported with tables with auto_increment column. + tk.MustExec("create table auto (id int not null auto_increment unique) shard_row_id_bits = 4") + tk.MustExec("alter table auto shard_row_id_bits = 5") + tk.MustExec("drop table auto") + tk.MustExec("create table auto (id int not null auto_increment unique) shard_row_id_bits = 0") + tk.MustExec("alter table auto shard_row_id_bits = 5") + tk.MustExec("drop table auto") + tk.MustExec("create table auto (id int not null auto_increment unique)") + tk.MustExec("alter table auto shard_row_id_bits = 5") + tk.MustExec("drop table auto") + tk.MustExec("create table auto (id int not null auto_increment unique) shard_row_id_bits = 4") + tk.MustExec("alter table auto shard_row_id_bits = 0") + tk.MustExec("drop table auto") + + // After PR 10759, shard_row_id_bits is not supported with pk_is_handle tables. + err = tk.ExecToErr("create table auto (id int not null auto_increment primary key, b int) shard_row_id_bits = 4") + c.Assert(err.Error(), Equals, "[ddl:207]unsupported shard_row_id_bits for table with primary key as row id.") + tk.MustExec("create table auto (id int not null auto_increment primary key, b int) shard_row_id_bits = 0") + err = tk.ExecToErr("alter table auto shard_row_id_bits = 5") + c.Assert(err.Error(), Equals, "[ddl:207]unsupported shard_row_id_bits for table with primary key as row id.") + tk.MustExec("alter table auto shard_row_id_bits = 0") + + // Hack an existing table with shard_row_id_bits and primary key as handle + db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test")) + c.Assert(ok, IsTrue) + tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto")) + tblInfo := tbl.Meta() + tblInfo.ShardRowIDBits = 5 + tblInfo.MaxShardRowIDBits = 5 + + kv.RunInNewTxn(s.store, false, func(txn kv.Transaction) error { + m := meta.NewMeta(txn) + _, err = m.GenSchemaVersion() + c.Assert(err, IsNil) + c.Assert(m.UpdateTable(db.ID, tblInfo), IsNil) + return nil }) + err = dom.Reload() c.Assert(err, IsNil) - c.Assert(count, Equals, 100) - c.Assert(hasShardedID, IsTrue) - // Test that audo_increment column can not use shard_row_id_bits. - _, err = tk.Exec("create table auto (id int not null auto_increment primary key) shard_row_id_bits = 4") - c.Assert(err, NotNil) - tk.MustExec("create table auto (id int not null auto_increment primary key) shard_row_id_bits = 0") - _, err = tk.Exec("alter table auto shard_row_id_bits = 4") - c.Assert(err, NotNil) + tk.MustExec("insert auto(b) values (1), (3), (5)") + tk.MustQuery("select id from auto order by id").Check(testkit.Rows("1", "2", "3")) + tk.MustExec("alter table auto shard_row_id_bits = 0") + tk.MustExec("drop table auto") + + // Test shard_row_id_bits with auto_increment column + tk.MustExec("create table auto (a int, b int auto_increment unique) shard_row_id_bits = 15") + for i := 0; i < 100; i++ { + tk.MustExec(fmt.Sprintf("insert auto(a) values (%d)", i)) + } + tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto")) + assertCountAndShard(tbl, 100) + prevB, err := strconv.Atoi(tk.MustQuery("select b from auto where a=0").Rows()[0][0].(string)) + c.Assert(err, IsNil) + for i := 1; i < 100; i++ { + b, err := strconv.Atoi(tk.MustQuery(fmt.Sprintf("select b from auto where a=%d", i)).Rows()[0][0].(string)) + c.Assert(err, IsNil) + c.Assert(b, Greater, prevB) + prevB = b + } // Test overflow tk.MustExec("drop table if exists t1") tk.MustExec("create table t1 (a int) shard_row_id_bits = 15") defer tk.MustExec("drop table if exists t1") - tbl, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + 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) diff --git a/executor/insert_common.go b/executor/insert_common.go index a9bbded3b6664..49ac8953b5319 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -501,7 +501,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(d types.Datum, hasValue bool, c // Change NULL to auto id. // Change value 0 to auto id, if NoAutoValueOnZero SQL mode is not set. if d.IsNull() || e.ctx.GetSessionVars().SQLMode&mysql.ModeNoAutoValueOnZero == 0 { - recordID, err = e.Table.AllocAutoID(e.ctx) + recordID, err = e.Table.AllocAutoIncrementValue(e.ctx) if e.filterErr(err) != nil { return types.Datum{}, err } diff --git a/infoschema/tables.go b/infoschema/tables.go index 697a68a7d1997..4a64164f69127 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -1870,6 +1870,7 @@ func (it *infoschemaTable) getRows(ctx sessionctx.Context, cols []*table.Column) return rows, nil } +// IterRecords implements table.Table IterRecords interface. func (it *infoschemaTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc) error { if len(startKey) != 0 { @@ -1891,6 +1892,7 @@ func (it *infoschemaTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, return nil } +// RowWithCols implements table.Table RowWithCols interface. func (it *infoschemaTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } @@ -1900,79 +1902,102 @@ func (it *infoschemaTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, return nil, table.ErrUnsupportedOp } +// Cols implements table.Table Cols interface. func (it *infoschemaTable) Cols() []*table.Column { return it.cols } +// WritableCols implements table.Table WritableCols interface. func (it *infoschemaTable) WritableCols() []*table.Column { return it.cols } +// Indices implements table.Table Indices interface. func (it *infoschemaTable) Indices() []table.Index { return nil } +// WritableIndices implements table.Table WritableIndices interface. func (it *infoschemaTable) WritableIndices() []table.Index { return nil } +// DeletableIndices implements table.Table DeletableIndices interface. func (it *infoschemaTable) DeletableIndices() []table.Index { return nil } +// RecordPrefix implements table.Table RecordPrefix interface. func (it *infoschemaTable) RecordPrefix() kv.Key { return nil } +// IndexPrefix implements table.Table IndexPrefix interface. func (it *infoschemaTable) IndexPrefix() kv.Key { return nil } +// FirstKey implements table.Table FirstKey interface. func (it *infoschemaTable) FirstKey() kv.Key { return nil } +// RecordKey implements table.Table RecordKey interface. func (it *infoschemaTable) RecordKey(h int64) kv.Key { return nil } +// AddRecord implements table.Table AddRecord interface. func (it *infoschemaTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) { return 0, table.ErrUnsupportedOp } +// RemoveRecord implements table.Table RemoveRecord interface. func (it *infoschemaTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { return table.ErrUnsupportedOp } +// UpdateRecord implements table.Table UpdateRecord interface. func (it *infoschemaTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { return table.ErrUnsupportedOp } -func (it *infoschemaTable) AllocAutoID(ctx sessionctx.Context) (int64, error) { +// AllocAutoIncrementValue implements table.Table AllocAutoIncrementValue interface. +func (it *infoschemaTable) AllocAutoIncrementValue(ctx sessionctx.Context) (int64, error) { return 0, table.ErrUnsupportedOp } +// AllocHandle implements table.Table AllocHandle interface. +func (it *infoschemaTable) AllocHandle(ctx sessionctx.Context) (int64, error) { + return 0, table.ErrUnsupportedOp +} + +// Allocator implements table.Table Allocator interface. func (it *infoschemaTable) Allocator(ctx sessionctx.Context) autoid.Allocator { return nil } +// RebaseAutoID implements table.Table RebaseAutoID interface. func (it *infoschemaTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { return table.ErrUnsupportedOp } +// Meta implements table.Table Meta interface. func (it *infoschemaTable) Meta() *model.TableInfo { return it.meta } +// GetPhysicalID implements table.Table GetPhysicalID interface. func (it *infoschemaTable) GetPhysicalID() int64 { return it.meta.ID } -// Seek is the first method called for table scan, we lazy initialize it here. +// Seek implements table.Table Seek interface. func (it *infoschemaTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { return 0, false, table.ErrUnsupportedOp } +// Type implements table.Table Type interface. func (it *infoschemaTable) Type() table.Type { return table.VirtualTable } @@ -1980,7 +2005,7 @@ func (it *infoschemaTable) Type() table.Type { // VirtualTable is a dummy table.Table implementation. type VirtualTable struct{} -// IterRecords implements table.Table Type interface. +// IterRecords implements table.Table IterRecords interface. func (vt *VirtualTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, cols []*table.Column, fn table.RecordIterFunc) error { if len(startKey) != 0 { @@ -1989,92 +2014,97 @@ func (vt *VirtualTable) IterRecords(ctx sessionctx.Context, startKey kv.Key, col return nil } -// RowWithCols implements table.Table Type interface. +// RowWithCols implements table.Table RowWithCols interface. func (vt *VirtualTable) RowWithCols(ctx sessionctx.Context, h int64, cols []*table.Column) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } -// Row implements table.Table Type interface. +// Row implements table.Table Row interface. func (vt *VirtualTable) Row(ctx sessionctx.Context, h int64) ([]types.Datum, error) { return nil, table.ErrUnsupportedOp } -// Cols implements table.Table Type interface. +// Cols implements table.Table Cols interface. func (vt *VirtualTable) Cols() []*table.Column { return nil } -// WritableCols implements table.Table Type interface. +// WritableCols implements table.Table WritableCols interface. func (vt *VirtualTable) WritableCols() []*table.Column { return nil } -// Indices implements table.Table Type interface. +// Indices implements table.Table Indices interface. func (vt *VirtualTable) Indices() []table.Index { return nil } -// WritableIndices implements table.Table Type interface. +// WritableIndices implements table.Table WritableIndices interface. func (vt *VirtualTable) WritableIndices() []table.Index { return nil } -// DeletableIndices implements table.Table Type interface. +// DeletableIndices implements table.Table DeletableIndices interface. func (vt *VirtualTable) DeletableIndices() []table.Index { return nil } -// RecordPrefix implements table.Table Type interface. +// RecordPrefix implements table.Table RecordPrefix interface. func (vt *VirtualTable) RecordPrefix() kv.Key { return nil } -// IndexPrefix implements table.Table Type interface. +// IndexPrefix implements table.Table IndexPrefix interface. func (vt *VirtualTable) IndexPrefix() kv.Key { return nil } -// FirstKey implements table.Table Type interface. +// FirstKey implements table.Table FirstKey interface. func (vt *VirtualTable) FirstKey() kv.Key { return nil } -// RecordKey implements table.Table Type interface. +// RecordKey implements table.Table RecordKey interface. func (vt *VirtualTable) RecordKey(h int64) kv.Key { return nil } -// AddRecord implements table.Table Type interface. +// AddRecord implements table.Table AddRecord interface. func (vt *VirtualTable) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ...*table.AddRecordOpt) (recordID int64, err error) { return 0, table.ErrUnsupportedOp } -// RemoveRecord implements table.Table Type interface. +// RemoveRecord implements table.Table RemoveRecord interface. func (vt *VirtualTable) RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error { return table.ErrUnsupportedOp } -// UpdateRecord implements table.Table Type interface. +// UpdateRecord implements table.Table UpdateRecord interface. func (vt *VirtualTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, newData []types.Datum, touched []bool) error { return table.ErrUnsupportedOp } -// AllocAutoID implements table.Table Type interface. -func (vt *VirtualTable) AllocAutoID(ctx sessionctx.Context) (int64, error) { +// AllocAutoIncrementValue implements table.Table AllocAutoIncrementValue interface. +func (vt *VirtualTable) AllocAutoIncrementValue(ctx sessionctx.Context) (int64, error) { + return 0, table.ErrUnsupportedOp +} + +// AllocHandle implements table.Table AllocHandle interface. +func (vt *VirtualTable) AllocHandle(ctx sessionctx.Context) (int64, error) { return 0, table.ErrUnsupportedOp } -// Allocator implements table.Table Type interface. +// Allocator implements table.Table Allocator interface. func (vt *VirtualTable) Allocator(ctx sessionctx.Context) autoid.Allocator { return nil } -// RebaseAutoID implements table.Table Type interface. +// RebaseAutoID implements table.Table RebaseAutoID interface. func (vt *VirtualTable) RebaseAutoID(ctx sessionctx.Context, newBase int64, isSetStep bool) error { return table.ErrUnsupportedOp } -// Meta implements table.Table Type interface. +// Meta implements table.Table Meta interface. func (vt *VirtualTable) Meta() *model.TableInfo { return nil } @@ -2084,7 +2114,7 @@ func (vt *VirtualTable) GetPhysicalID() int64 { return 0 } -// Seek implements table.Table Type interface. +// Seek implements table.Table Seek interface. func (vt *VirtualTable) Seek(ctx sessionctx.Context, h int64) (int64, bool, error) { return 0, false, table.ErrUnsupportedOp } diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index f960d14ef95c4..86c20c87751ba 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -129,8 +129,8 @@ func (s *testBinlogSuite) TestBinlog(c *C) { tk.Se.GetSessionVars().BinlogClient = s.client pump := s.pump tk.MustExec("drop table if exists local_binlog") - ddlQuery := "create table local_binlog (id int primary key, name varchar(10)) shard_row_id_bits=1" - binlogDDLQuery := "create table local_binlog (id int primary key, name varchar(10)) /*!90000 shard_row_id_bits=1 */" + ddlQuery := "create table local_binlog (id int unique key, name varchar(10)) shard_row_id_bits=1" + binlogDDLQuery := "create table local_binlog (id int unique key, name varchar(10)) /*!90000 shard_row_id_bits=1 */" tk.MustExec(ddlQuery) var matched bool // got matched pre DDL and commit DDL for i := 0; i < 10; i++ { @@ -155,7 +155,7 @@ func (s *testBinlogSuite) TestBinlog(c *C) { {types.NewIntDatum(1), types.NewStringDatum("abc")}, {types.NewIntDatum(2), types.NewStringDatum("cde")}, } - gotRows := mutationRowsToRows(c, prewriteVal.Mutations[0].InsertedRows, 0, 2) + gotRows := mutationRowsToRows(c, prewriteVal.Mutations[0].InsertedRows, 2, 4) c.Assert(gotRows, DeepEquals, expected) tk.MustExec("update local_binlog set name = 'xyz' where id = 2") @@ -169,7 +169,7 @@ func (s *testBinlogSuite) TestBinlog(c *C) { gotRows = mutationRowsToRows(c, prewriteVal.Mutations[0].UpdatedRows, 1, 3) c.Assert(gotRows, DeepEquals, oldRow) - gotRows = mutationRowsToRows(c, prewriteVal.Mutations[0].UpdatedRows, 5, 7) + gotRows = mutationRowsToRows(c, prewriteVal.Mutations[0].UpdatedRows, 7, 9) c.Assert(gotRows, DeepEquals, newRow) tk.MustExec("delete from local_binlog where id = 1") diff --git a/table/table.go b/table/table.go index b0952a6c46096..2fba027e6df1c 100644 --- a/table/table.go +++ b/table/table.go @@ -143,8 +143,11 @@ type Table interface { // RemoveRecord removes a row in the table. RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error - // AllocAutoID allocates an auto_increment ID for a new row. - AllocAutoID(ctx sessionctx.Context) (int64, error) + // AllocAutoIncrementValue allocates an auto_increment value for a new row. + AllocAutoIncrementValue(ctx sessionctx.Context) (int64, error) + + // AllocHandle allocates a handle for a new row. + AllocHandle(ctx sessionctx.Context) (int64, error) // Allocator returns Allocator. Allocator(ctx sessionctx.Context) autoid.Allocator diff --git a/table/tables/tables.go b/table/tables/tables.go index 3b1497345ead1..d7ed18fbe3f97 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -443,7 +443,7 @@ func (t *tableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts .. } } if !hasRecordID { - recordID, err = t.AllocAutoID(ctx) + recordID, err = t.AllocHandle(ctx) if err != nil { return 0, err } @@ -914,8 +914,13 @@ func GetColDefaultValue(ctx sessionctx.Context, col *table.Column, defaultVals [ return colVal, nil } -// AllocAutoID implements table.Table AllocAutoID interface. -func (t *tableCommon) AllocAutoID(ctx sessionctx.Context) (int64, error) { +// AllocAutoIncrementValue implements table.Table AllocAutoIncrementValue interface. +func (t *tableCommon) AllocAutoIncrementValue(ctx sessionctx.Context) (int64, error) { + return t.Allocator(ctx).Alloc(t.tableID) +} + +// AllocHandle implements table.Table AllocHandle interface. +func (t *tableCommon) AllocHandle(ctx sessionctx.Context) (int64, error) { rowID, err := t.Allocator(ctx).Alloc(t.tableID) if err != nil { return 0, err diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index 30e6eac0bb526..16904d69e49ce 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -95,10 +95,14 @@ func (ts *testSuite) TestBasic(c *C) { c.Assert(string(tb.RecordPrefix()), Not(Equals), "") c.Assert(tables.FindIndexByColName(tb, "b"), NotNil) - autoid, err := tb.AllocAutoID(nil) + autoid, err := tb.AllocAutoIncrementValue(nil) c.Assert(err, IsNil) c.Assert(autoid, Greater, int64(0)) + handle, err := tb.AllocHandle(nil) + c.Assert(err, IsNil) + c.Assert(handle, Greater, int64(0)) + ctx := ts.se rid, err := tb.AddRecord(ctx, types.MakeDatums(1, "abc")) c.Assert(err, IsNil) @@ -239,10 +243,15 @@ func (ts *testSuite) TestUniqueIndexMultipleNullEntries(c *C) { c.Assert(string(tb.RecordPrefix()), Not(Equals), "") c.Assert(tables.FindIndexByColName(tb, "b"), NotNil) - autoid, err := tb.AllocAutoID(nil) - sctx := ts.se + handle, err := tb.AllocHandle(nil) + c.Assert(err, IsNil) + c.Assert(handle, Greater, int64(0)) + + autoid, err := tb.AllocAutoIncrementValue(nil) c.Assert(err, IsNil) c.Assert(autoid, Greater, int64(0)) + + sctx := ts.se c.Assert(sctx.NewTxn(ctx), IsNil) _, err = tb.AddRecord(sctx, types.MakeDatums(1, nil)) c.Assert(err, IsNil) @@ -373,13 +382,13 @@ func (ts *testSuite) TestTableFromMeta(c *C) { tk.MustExec("create table t_meta (a int) shard_row_id_bits = 15") tb, err = domain.GetDomain(tk.Se).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t_meta")) c.Assert(err, IsNil) - _, err = tb.AllocAutoID(tk.Se) + _, err = tb.AllocHandle(tk.Se) c.Assert(err, IsNil) maxID := 1<<(64-15-1) - 1 err = tb.RebaseAutoID(tk.Se, int64(maxID), false) c.Assert(err, IsNil) - _, err = tb.AllocAutoID(tk.Se) + _, err = tb.AllocHandle(tk.Se) c.Assert(err, NotNil) }