Skip to content

Commit

Permalink
*: refine the table.Table interface (pingcap#15227)
Browse files Browse the repository at this point in the history
remove unnecessary methods.
"The bigger the interface, the weaker the abstraction -- Rob Pike"
  • Loading branch information
tiancaiamao authored Mar 10, 2020
1 parent bd685ca commit b504d37
Show file tree
Hide file tree
Showing 13 changed files with 46 additions and 86 deletions.
2 changes: 1 addition & 1 deletion ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -2096,7 +2096,7 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6
if err != nil {
return errors.Trace(err)
}
autoIncID, err := t.Allocator(ctx, autoid.RowIDAllocType).NextGlobalAutoID(t.Meta().ID)
autoIncID, err := t.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(t.Meta().ID)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -557,7 +557,7 @@ func verifyNoOverflowShardBits(s *sessionPool, tbl table.Table, shardRowIDBits u
defer s.put(ctx)

// Check next global max auto ID first.
autoIncID, err := tbl.Allocator(ctx, autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID)
autoIncID, err := tbl.Allocators(ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID)
if err != nil {
return errors.Trace(err)
}
Expand Down
2 changes: 1 addition & 1 deletion executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -266,7 +266,7 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error {
break
}
}
nextGlobalID, err := tbl.Allocator(e.ctx, autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID)
nextGlobalID, err := tbl.Allocators(e.ctx).Get(autoid.RowIDAllocType).NextGlobalAutoID(tbl.Meta().ID)
if err != nil {
return err
}
Expand Down
2 changes: 1 addition & 1 deletion executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -235,7 +235,7 @@ func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *m
if err != nil {
return 0, err
}
return tbl.Allocator(ctx, autoid.RowIDAllocType).Base() + 1, nil
return tbl.Allocators(ctx).Get(autoid.RowIDAllocType).Base() + 1, nil
}

func (e *memtableRetriever) setDataFromSchemata(ctx sessionctx.Context, schemas []*model.DBInfo) {
Expand Down
4 changes: 2 additions & 2 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -901,7 +901,7 @@ func (e *InsertValues) adjustAutoRandomDatum(ctx context.Context, d types.Datum,

// allocAutoRandomID allocates a random id for primary key column. It assumes tableInfo.AutoRandomBits > 0.
func (e *InsertValues) allocAutoRandomID(fieldType *types.FieldType) (int64, error) {
alloc := e.Table.Allocator(e.ctx, autoid.AutoRandomType)
alloc := e.Table.Allocators(e.ctx).Get(autoid.AutoRandomType)
tableInfo := e.Table.Meta()
_, autoRandomID, err := alloc.Alloc(tableInfo.ID, 1, 1, 1)
if err != nil {
Expand All @@ -918,7 +918,7 @@ func (e *InsertValues) allocAutoRandomID(fieldType *types.FieldType) (int64, err
}

func (e *InsertValues) rebaseAutoRandomID(recordID int64, fieldType *types.FieldType) error {
alloc := e.Table.Allocator(e.ctx, autoid.AutoRandomType)
alloc := e.Table.Allocators(e.ctx).Get(autoid.AutoRandomType)
tableInfo := e.Table.Meta()

typeBitsLength := uint64(mysql.DefaultLengthOfMysqlTypes[fieldType.Tp] * 8)
Expand Down
2 changes: 1 addition & 1 deletion executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -943,7 +943,7 @@ func (e *ShowExec) fetchShowCreateTable() error {
}

tableInfo := tb.Meta()
allocator := tb.Allocator(e.ctx, autoid.RowIDAllocType)
allocator := tb.Allocators(e.ctx).Get(autoid.RowIDAllocType)
var buf bytes.Buffer
// TODO: let the result more like MySQL.
if err = ConstructResultOfShowCreateTable(e.ctx, tableInfo, allocator, &buf); err != nil {
Expand Down
2 changes: 1 addition & 1 deletion executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -219,7 +219,7 @@ func rebaseAutoRandomValue(sctx sessionctx.Context, t table.Table, newData *type
}
shardBits := tableInfo.AutoRandomBits + 1 // sign bit is reserved.
recordID = recordID << shardBits >> shardBits
return t.Allocator(sctx, autoid.AutoRandomType).Rebase(tableInfo.ID, recordID, true)
return t.Allocators(sctx).Get(autoid.AutoRandomType).Rebase(tableInfo.ID, recordID, true)
}

// resetErrDataTooLong reset ErrDataTooLong error msg.
Expand Down
2 changes: 1 addition & 1 deletion infoschema/infoschema.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,7 +251,7 @@ func (is *infoSchema) AllocByID(id int64) (autoid.Allocators, bool) {
if !ok {
return nil, false
}
return tbl.AllAllocators(nil), true
return tbl.Allocators(nil), true
}

func (is *infoSchema) AllSchemaNames() (names []string) {
Expand Down
38 changes: 4 additions & 34 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -2105,23 +2105,8 @@ func (it *infoschemaTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData
return table.ErrUnsupportedOp
}

// AllocHandle implements table.Table AllocHandle interface.
func (it *infoschemaTable) AllocHandle(ctx sessionctx.Context) (int64, error) {
return 0, table.ErrUnsupportedOp
}

// AllocHandleIDs implements table.Table AllocHandleIDs interface.
func (it *infoschemaTable) AllocHandleIDs(ctx sessionctx.Context, n uint64) (int64, int64, error) {
return 0, 0, table.ErrUnsupportedOp
}

// Allocator implements table.Table Allocator interface.
func (it *infoschemaTable) Allocator(_ sessionctx.Context, _ autoid.AllocatorType) autoid.Allocator {
return nil
}

// AllAllocators implements table.Table AllAllocators interface.
func (it *infoschemaTable) AllAllocators(_ sessionctx.Context) autoid.Allocators {
// Allocators implements table.Table Allocators interface.
func (it *infoschemaTable) Allocators(_ sessionctx.Context) autoid.Allocators {
return nil
}

Expand Down Expand Up @@ -2247,23 +2232,8 @@ func (vt *VirtualTable) UpdateRecord(ctx sessionctx.Context, h int64, oldData, n
return table.ErrUnsupportedOp
}

// AllocHandle implements table.Table AllocHandle interface.
func (vt *VirtualTable) AllocHandle(ctx sessionctx.Context) (int64, error) {
return 0, table.ErrUnsupportedOp
}

// AllocHandleIDs implements table.Table AllocHandleIDs interface.
func (vt *VirtualTable) AllocHandleIDs(ctx sessionctx.Context, n uint64) (int64, int64, error) {
return 0, 0, table.ErrUnsupportedOp
}

// Allocator implements table.Table Allocator interface.
func (vt *VirtualTable) Allocator(_ sessionctx.Context, _ autoid.AllocatorType) autoid.Allocator {
return nil
}

// AllAllocators implements table.Table AllAllocators interface.
func (vt *VirtualTable) AllAllocators(_ sessionctx.Context) autoid.Allocators {
// Allocators implements table.Table Allocators interface.
func (vt *VirtualTable) Allocators(_ sessionctx.Context) autoid.Allocators {
return nil
}

Expand Down
10 changes: 10 additions & 0 deletions meta/autoid/autoid.go
Original file line number Diff line number Diff line change
Expand Up @@ -116,6 +116,16 @@ func NewAllocators(allocators ...Allocator) Allocators {
return allocators
}

// Get returns the Allocator according to the AllocatorType.
func (all Allocators) Get(allocType AllocatorType) Allocator {
for _, a := range all {
if a.GetType() == allocType {
return a
}
}
return nil
}

type allocator struct {
mu sync.Mutex
base int64
Expand Down
17 changes: 4 additions & 13 deletions table/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -191,17 +191,8 @@ type Table interface {
// RemoveRecord removes a row in the table.
RemoveRecord(ctx sessionctx.Context, h int64, r []types.Datum) error

// AllocHandle allocates a handle for a new row.
AllocHandle(ctx sessionctx.Context) (int64, error)

// AllocHandleIDs allocates multiple handle for rows.
AllocHandleIDs(ctx sessionctx.Context, n uint64) (int64, int64, error)

// Allocator returns Allocator.
Allocator(ctx sessionctx.Context, allocatorType autoid.AllocatorType) autoid.Allocator

// AllAllocators returns all allocators.
AllAllocators(ctx sessionctx.Context) autoid.Allocators
// Allocators returns all allocators.
Allocators(ctx sessionctx.Context) autoid.Allocators

// RebaseAutoID rebases the auto_increment ID base.
// If allocIDs is true, it will allocate some IDs and save to the cache.
Expand All @@ -226,7 +217,7 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte
}
increment := sctx.GetSessionVars().AutoIncrementIncrement
offset := sctx.GetSessionVars().AutoIncrementOffset
_, max, err := t.Allocator(sctx, autoid.RowIDAllocType).Alloc(t.Meta().ID, uint64(1), int64(increment), int64(offset))
_, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(t.Meta().ID, uint64(1), int64(increment), int64(offset))
if err != nil {
return 0, err
}
Expand All @@ -242,7 +233,7 @@ func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.
}
increment = int64(sctx.GetSessionVars().AutoIncrementIncrement)
offset := int64(sctx.GetSessionVars().AutoIncrementOffset)
min, max, err := t.Allocator(sctx, autoid.RowIDAllocType).Alloc(t.Meta().ID, uint64(N), increment, offset)
min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(t.Meta().ID, uint64(N), increment, offset)
if err != nil {
return min, max, err
}
Expand Down
39 changes: 14 additions & 25 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -491,15 +491,15 @@ func (t *TableCommon) AddRecord(ctx sessionctx.Context, r []types.Datum, opts ..
rows := stmtCtx.RecordRows()
if rows > 1 {
if stmtCtx.BaseRowID >= stmtCtx.MaxRowID {
stmtCtx.BaseRowID, stmtCtx.MaxRowID, err = t.AllocHandleIDs(ctx, rows)
stmtCtx.BaseRowID, stmtCtx.MaxRowID, err = allocHandleIDs(ctx, t, rows)
if err != nil {
return 0, err
}
}
stmtCtx.BaseRowID += 1
recordID = stmtCtx.BaseRowID
} else {
recordID, err = t.AllocHandle(ctx)
recordID, err = AllocHandle(ctx, t)
if err != nil {
return 0, err
}
Expand Down Expand Up @@ -1017,21 +1017,21 @@ func GetColDefaultValue(ctx sessionctx.Context, col *table.Column, defaultVals [
return colVal, nil
}

// AllocHandle implements table.Table AllocHandle interface.
func (t *TableCommon) AllocHandle(ctx sessionctx.Context) (int64, error) {
_, rowID, err := t.AllocHandleIDs(ctx, 1)
// AllocHandle allocate a new handle.
func AllocHandle(ctx sessionctx.Context, t table.Table) (int64, error) {
_, rowID, err := allocHandleIDs(ctx, t, 1)
return rowID, err
}

// AllocHandleIDs implements table.Table AllocHandleIDs interface.
func (t *TableCommon) AllocHandleIDs(ctx sessionctx.Context, n uint64) (int64, int64, error) {
base, maxID, err := t.Allocator(ctx, autoid.RowIDAllocType).Alloc(t.tableID, n, 1, 1)
func allocHandleIDs(ctx sessionctx.Context, t table.Table, n uint64) (int64, int64, error) {
meta := t.Meta()
base, maxID, err := t.Allocators(ctx).Get(autoid.RowIDAllocType).Alloc(meta.ID, n, 1, 1)
if err != nil {
return 0, 0, err
}
if t.meta.ShardRowIDBits > 0 {
if meta.ShardRowIDBits > 0 {
// Use max record ShardRowIDBits to check overflow.
if OverflowShardBits(maxID, t.meta.MaxShardRowIDBits, autoid.RowIDBitLength) {
if OverflowShardBits(maxID, meta.MaxShardRowIDBits, autoid.RowIDBitLength) {
// If overflow, the rowID may be duplicated. For examples,
// t.meta.ShardRowIDBits = 4
// rowID = 0010111111111111111111111111111111111111111111111111111111111111
Expand All @@ -1043,7 +1043,7 @@ func (t *TableCommon) AllocHandleIDs(ctx sessionctx.Context, n uint64) (int64, i
}
txnCtx := ctx.GetSessionVars().TxnCtx
if txnCtx.Shard == nil {
shard := CalcShard(t.meta.ShardRowIDBits, txnCtx.StartTS, autoid.RowIDBitLength)
shard := CalcShard(meta.ShardRowIDBits, txnCtx.StartTS, autoid.RowIDBitLength)
txnCtx.Shard = &shard
}
base |= *txnCtx.Shard
Expand All @@ -1066,19 +1066,8 @@ func CalcShard(shardRowIDBits uint64, startTS uint64, typeBitsLength uint64) int
return (hashVal & (1<<shardRowIDBits - 1)) << (typeBitsLength - shardRowIDBits - 1)
}

// Allocator implements table.Table Allocator interface.
func (t *TableCommon) Allocator(ctx sessionctx.Context, allocType autoid.AllocatorType) autoid.Allocator {
allAllocs := t.AllAllocators(ctx)
for _, a := range allAllocs {
if a.GetType() == allocType {
return a
}
}
return nil
}

// AllAllocators implements table.Table AllAllocators interface.
func (t *TableCommon) AllAllocators(ctx sessionctx.Context) autoid.Allocators {
// Allocators implements table.Table Allocators interface.
func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators {
if ctx == nil || ctx.GetSessionVars().IDAllocator == nil {
return t.allocs
}
Expand All @@ -1104,7 +1093,7 @@ func (t *TableCommon) AllAllocators(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.Allocator(ctx, autoid.RowIDAllocType).Rebase(t.tableID, newBase, isSetStep)
return t.Allocators(ctx).Get(autoid.RowIDAllocType).Rebase(t.tableID, newBase, isSetStep)
}

// Seek implements table.Table Seek interface.
Expand Down
10 changes: 5 additions & 5 deletions table/tables/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ func (ts *testSuite) TestBasic(c *C) {
c.Assert(err, IsNil)
c.Assert(autoID, Greater, int64(0))

handle, err := tb.AllocHandle(nil)
handle, err := tables.AllocHandle(nil, tb)
c.Assert(err, IsNil)
c.Assert(handle, Greater, int64(0))

Expand Down Expand Up @@ -158,7 +158,7 @@ func (ts *testSuite) TestBasic(c *C) {
c.Assert(err, IsNil)

table.MockTableFromMeta(tb.Meta())
alc := tb.Allocator(nil, autoid.RowIDAllocType)
alc := tb.Allocators(nil).Get(autoid.RowIDAllocType)
c.Assert(alc, NotNil)

err = tb.RebaseAutoID(nil, 0, false)
Expand Down Expand Up @@ -245,7 +245,7 @@ func (ts *testSuite) TestUniqueIndexMultipleNullEntries(c *C) {
c.Assert(string(tb.RecordPrefix()), Not(Equals), "")
c.Assert(tables.FindIndexByColName(tb, "b"), NotNil)

handle, err := tb.AllocHandle(nil)
handle, err := tables.AllocHandle(nil, tb)
c.Assert(err, IsNil)
c.Assert(handle, Greater, int64(0))

Expand Down Expand Up @@ -384,14 +384,14 @@ 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.AllocHandle(tk.Se)
_, err = tables.AllocHandle(tk.Se, tb)
c.Assert(err, IsNil)

maxID := 1<<(64-15-1) - 1
err = tb.RebaseAutoID(tk.Se, int64(maxID), false)
c.Assert(err, IsNil)

_, err = tb.AllocHandle(tk.Se)
_, err = tables.AllocHandle(tk.Se, tb)
c.Assert(err, NotNil)
}

Expand Down

0 comments on commit b504d37

Please sign in to comment.