Skip to content

Commit

Permalink
Merge remote-tracking branch 'upstream/master' into collate
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 committed Mar 10, 2020
2 parents 2475837 + 897a197 commit bbadebe
Show file tree
Hide file tree
Showing 23 changed files with 89 additions and 137 deletions.
3 changes: 3 additions & 0 deletions .github/labeler.yml
Original file line number Diff line number Diff line change
Expand Up @@ -26,3 +26,6 @@ component/DDL:
- ddl/*
- infoschema/*
- meta/*

component/config:
- config/*
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
8 changes: 3 additions & 5 deletions kv/memdb/memdb.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,8 @@ import (
"bytes"
"math"
"unsafe"

"github.com/pingcap/tidb/util/fastrand"
)

const (
Expand Down Expand Up @@ -315,13 +317,9 @@ func (db *DB) newNode(arena *arena, key []byte, v []byte, height int) (*node, ar
return node, addr
}

// fastRand is a fast thread local random function.
//go:linkname fastRand runtime.fastrand
func fastRand() uint32

func (db *DB) randomHeight() int {
h := 1
for h < maxHeight && fastRand() < uint32(math.MaxUint32)/4 {
for h < maxHeight && fastrand.Uint32() < uint32(math.MaxUint32)/4 {
h++
}
return h
Expand Down
12 changes: 0 additions & 12 deletions kv/memdb/memdb.s

This file was deleted.

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
4 changes: 3 additions & 1 deletion server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@ import (
"net"
"net/http"
"unsafe"

// For pprof
_ "net/http/pprof"
"os"
Expand All @@ -55,6 +56,7 @@ import (
"github.com/pingcap/tidb/plugin"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/fastrand"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/sys/linux"
"github.com/pingcap/tidb/util/timeutil"
Expand Down Expand Up @@ -158,7 +160,7 @@ func (s *Server) newConn(conn net.Conn) *clientConn {
}
}
cc.setConn(conn)
cc.salt = util.RandomBuf(20)
cc.salt = fastrand.Buf(20)
return cc
}

Expand Down
6 changes: 3 additions & 3 deletions statistics/sample.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,8 +23,8 @@ import (
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/fastrand"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tipb/go-tipb"
"github.com/spaolacci/murmur3"
Expand Down Expand Up @@ -166,9 +166,9 @@ func (c *SampleCollector) collect(sc *stmtctx.StatementContext, d types.Datum) e
d.Copy(&newItem.Value)
c.Samples = append(c.Samples, newItem)
} else {
shouldAdd := int64(util.FastRand64N(uint64(c.seenValues))) < c.MaxSampleSize
shouldAdd := int64(fastrand.Uint64N(uint64(c.seenValues))) < c.MaxSampleSize
if shouldAdd {
idx := int(util.FastRand32N(uint32(c.MaxSampleSize)))
idx := int(fastrand.Uint32N(uint32(c.MaxSampleSize)))
newItem := &SampleItem{}
d.Copy(&newItem.Value)
// To keep the order of the elements, we use delete and append, not direct replacement.
Expand Down
8 changes: 4 additions & 4 deletions store/tikv/backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,7 +25,7 @@ import (
"github.com/pingcap/log"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/fastrand"
"github.com/pingcap/tidb/util/logutil"
"github.com/prometheus/client_golang/prometheus"
"go.uber.org/zap"
Expand Down Expand Up @@ -91,12 +91,12 @@ func NewBackoffFn(base, cap, jitter int) func(ctx context.Context, maxSleepMs in
sleep = expo(base, cap, attempts)
case FullJitter:
v := expo(base, cap, attempts)
sleep = int(util.FastRand32N(uint32(v)))
sleep = int(fastrand.Uint32N(uint32(v)))
case EqualJitter:
v := expo(base, cap, attempts)
sleep = v/2 + int(util.FastRand32N(uint32(v/2)))
sleep = v/2 + int(fastrand.Uint32N(uint32(v/2)))
case DecorrJitter:
sleep = int(math.Min(float64(cap), float64(base+int(util.FastRand32N(uint32(lastSleep*3-base))))))
sleep = int(math.Min(float64(cap), float64(base+int(fastrand.Uint32N(uint32(lastSleep*3-base))))))
}
logutil.BgLogger().Debug("backoff",
zap.Int("base", base),
Expand Down
4 changes: 2 additions & 2 deletions store/tikv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -33,7 +33,7 @@ import (
"github.com/pingcap/tidb/store/tikv/oracle"
"github.com/pingcap/tidb/store/tikv/oracle/oracles"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/fastrand"
"github.com/pingcap/tidb/util/logutil"
"go.etcd.io/etcd/clientv3"
"go.uber.org/zap"
Expand Down Expand Up @@ -209,7 +209,7 @@ func newTikvStore(uuid string, pdClient pd.Client, spkv SafePointKV, client Clie
safePoint: 0,
spTime: time.Now(),
closed: make(chan struct{}),
replicaReadSeed: util.FastRand(),
replicaReadSeed: fastrand.Uint32(),
}
store.lockResolver = newLockResolver(store)
store.enableGC = enableGC
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
Loading

0 comments on commit bbadebe

Please sign in to comment.