Skip to content

Commit

Permalink
meta: make auto increment id can be adjust. (pingcap#10978) (pingcap#…
Browse files Browse the repository at this point in the history
  • Loading branch information
Lingyu Song authored and jackysp committed Jul 1, 2019
1 parent b65f81f commit 8d757ee
Show file tree
Hide file tree
Showing 5 changed files with 90 additions and 15 deletions.
9 changes: 9 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,6 +25,7 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
"github.com/pingcap/parser/charset"
"github.com/pingcap/parser/model"
Expand Down Expand Up @@ -2850,6 +2851,10 @@ func (s *testDBSuite) TestComment(c *C) {
}

func (s *testDBSuite) TestRebaseAutoID(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil)
}()
s.tk = testkit.NewTestKit(c, s.store)
s.tk.MustExec("use " + s.schemaName)

Expand Down Expand Up @@ -4657,6 +4662,10 @@ func (s *testDBSuite) TestCanceledJobTakeTime(c *C) {
}

func (s *testDBSuite) TestAlterShardRowIDBits(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil)
}()
s.tk = testkit.NewTestKit(c, s.store)
tk := s.tk

Expand Down
5 changes: 5 additions & 0 deletions executor/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"fmt"

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/meta/autoid"
Expand Down Expand Up @@ -536,6 +537,10 @@ func (s *testSuite) TestAdminCheckPrimaryIndex(c *C) {
}

func (s *testSuite) TestAdminShowNextID(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil)
}()
step := int64(10)
autoIDStep := autoid.GetStep()
autoid.SetStep(step)
Expand Down
5 changes: 5 additions & 0 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (
"time"

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
Expand Down Expand Up @@ -260,6 +261,10 @@ func (s *testSuite) TestDefaultDBAfterDropCurDB(c *C) {
}

func (s *testSuite) TestRenameTable(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil)
}()
tk := testkit.NewTestKit(c, s.store)

tk.MustExec("create database rename1")
Expand Down
65 changes: 50 additions & 15 deletions meta/autoid/autoid.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (

"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
Expand All @@ -30,6 +31,12 @@ import (
"go.uber.org/zap"
)

const (
minStep = 1000
maxStep = 2000000
defaultConsumeTime = 10 * time.Second
)

// Test needs to change it, so it's a variable.
var step = int64(30000)

Expand Down Expand Up @@ -59,8 +66,10 @@ type allocator struct {
end int64
store kv.Storage
// dbID is current database's ID.
dbID int64
isUnsigned bool
dbID int64
isUnsigned bool
lastAllocTime time.Time
step int64
}

// GetStep is only used by tests
Expand Down Expand Up @@ -121,7 +130,7 @@ func (alloc *allocator) rebase4Unsigned(tableID int64, requiredBase uint64, allo
uCurrentEnd := uint64(currentEnd)
if allocIDs {
newBase = mathutil.MaxUint64(uCurrentEnd, requiredBase)
newEnd = mathutil.MinUint64(math.MaxUint64-uint64(step), newBase) + uint64(step)
newEnd = mathutil.MinUint64(math.MaxUint64-uint64(alloc.step), newBase) + uint64(alloc.step)
} else {
if uCurrentEnd >= requiredBase {
newBase = uCurrentEnd
Expand Down Expand Up @@ -166,7 +175,7 @@ func (alloc *allocator) rebase4Signed(tableID, requiredBase int64, allocIDs bool
}
if allocIDs {
newBase = mathutil.MaxInt64(currentEnd, requiredBase)
newEnd = mathutil.MinInt64(math.MaxInt64-step, newBase) + step
newEnd = mathutil.MinInt64(math.MaxInt64-alloc.step, newBase) + alloc.step
} else {
if currentEnd >= requiredBase {
newBase = currentEnd
Expand Down Expand Up @@ -212,21 +221,24 @@ func (alloc *allocator) alloc4Unsigned(tableID int64) (int64, error) {
if alloc.base == alloc.end { // step
var newBase, newEnd int64
startTime := time.Now()
consumeDur := startTime.Sub(alloc.lastAllocTime)
alloc.step = NextStep(alloc.step, consumeDur)
err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error {
m := meta.NewMeta(txn)
var err1 error
newBase, err1 = m.GetAutoTableID(alloc.dbID, tableID)
if err1 != nil {
return errors.Trace(err1)
}
tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(step)))
tmpStep := int64(mathutil.MinUint64(math.MaxUint64-uint64(newBase), uint64(alloc.step)))
newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep)
return err1
})
metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds())
if err != nil {
return 0, err
}
alloc.lastAllocTime = time.Now()
if uint64(newBase) == math.MaxUint64 {
return 0, ErrAutoincReadFailed
}
Expand All @@ -245,21 +257,24 @@ func (alloc *allocator) alloc4Signed(tableID int64) (int64, error) {
if alloc.base == alloc.end { // step
var newBase, newEnd int64
startTime := time.Now()
consumeDur := startTime.Sub(alloc.lastAllocTime)
alloc.step = NextStep(alloc.step, consumeDur)
err := kv.RunInNewTxn(alloc.store, true, func(txn kv.Transaction) error {
m := meta.NewMeta(txn)
var err1 error
newBase, err1 = m.GetAutoTableID(alloc.dbID, tableID)
if err1 != nil {
return errors.Trace(err1)
}
tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, step)
tmpStep := mathutil.MinInt64(math.MaxInt64-newBase, alloc.step)
newEnd, err1 = m.GenAutoTableID(alloc.dbID, tableID, tmpStep)
return err1
})
metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDAlloc, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds())
if err != nil {
return 0, err
}
alloc.lastAllocTime = time.Now()
if newBase == math.MaxInt64 {
return 0, ErrAutoincReadFailed
}
Expand Down Expand Up @@ -287,6 +302,35 @@ func (alloc *allocator) Alloc(tableID int64) (int64, error) {
return alloc.alloc4Signed(tableID)
}

// NextStep return new auto id step according to previous step and consuming time.
func NextStep(curStep int64, consumeDur time.Duration) int64 {
failpoint.Inject("mockAutoIDChange", func(val failpoint.Value) {
if val.(bool) {
failpoint.Return(step)
}
})

consumeRate := defaultConsumeTime.Seconds() / consumeDur.Seconds()
res := int64(float64(curStep) * consumeRate)
if res < minStep {
return minStep
} else if res > maxStep {
return maxStep
}
return res
}

// NewAllocator returns a new auto increment id generator on the store.
func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool) Allocator {
return &allocator{
store: store,
dbID: dbID,
isUnsigned: isUnsigned,
step: step,
lastAllocTime: time.Now(),
}
}

var (
memID int64
memIDLock sync.Mutex
Expand Down Expand Up @@ -340,15 +384,6 @@ func (alloc *memoryAllocator) Alloc(tableID int64) (int64, error) {
return alloc.base, nil
}

// NewAllocator returns a new auto increment id generator on the store.
func NewAllocator(store kv.Storage, dbID int64, isUnsigned bool) Allocator {
return &allocator{
store: store,
dbID: dbID,
isUnsigned: isUnsigned,
}
}

// NewMemoryAllocator returns a new auto increment id generator in memory.
func NewMemoryAllocator(dbID int64) Allocator {
return &memoryAllocator{
Expand Down
21 changes: 21 additions & 0 deletions meta/autoid/autoid_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
Expand All @@ -39,6 +40,11 @@ type testSuite struct {
}

func (*testSuite) TestT(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil)
}()

store, err := mockstore.NewMockTikvStore()
c.Assert(err, IsNil)
defer store.Close()
Expand Down Expand Up @@ -130,6 +136,11 @@ func (*testSuite) TestT(c *C) {
}

func (*testSuite) TestUnsignedAutoid(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange", `return(true)`), IsNil)
defer func() {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/meta/autoid/mockAutoIDChange"), IsNil)
}()

store, err := mockstore.NewMockTikvStore()
c.Assert(err, IsNil)
defer store.Close()
Expand Down Expand Up @@ -315,3 +326,13 @@ func (*testSuite) TestRollbackAlloc(c *C) {
c.Assert(alloc.Base(), Equals, int64(0))
c.Assert(alloc.End(), Equals, int64(0))
}

// TestNextStep tests generate next auto id step.
func (*testSuite) TestNextStep(c *C) {
nextStep := autoid.NextStep(2000000, 1*time.Nanosecond)
c.Assert(nextStep, Equals, int64(2000000))
nextStep = autoid.NextStep(678910, 10*time.Second)
c.Assert(nextStep, Equals, int64(678910))
nextStep = autoid.NextStep(50000, 10*time.Minute)
c.Assert(nextStep, Equals, int64(1000))
}

0 comments on commit 8d757ee

Please sign in to comment.