Skip to content

Commit

Permalink
Merge branch 'release-2.1' into release-2.1
Browse files Browse the repository at this point in the history
  • Loading branch information
zz-jason authored Jul 1, 2019
2 parents 67b95bb + d8defc5 commit 28f12b5
Show file tree
Hide file tree
Showing 14 changed files with 182 additions and 31 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
15 changes: 13 additions & 2 deletions domain/global_vars_cache.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,9 @@ type GlobalVariableCache struct {
lastModify time.Time
rows []chunk.Row
fields []*ast.ResultField

// Unit test may like to disable it.
disable bool
}

const globalVariableCacheExpiry time.Duration = 2 * time.Second
Expand All @@ -44,14 +47,22 @@ func (gvc *GlobalVariableCache) Update(rows []chunk.Row, fields []*ast.ResultFie
func (gvc *GlobalVariableCache) Get() (succ bool, rows []chunk.Row, fields []*ast.ResultField) {
gvc.RLock()
defer gvc.RUnlock()
if time.Now().Sub(gvc.lastModify) < globalVariableCacheExpiry {
succ, rows, fields = true, gvc.rows, gvc.fields
if time.Since(gvc.lastModify) < globalVariableCacheExpiry {
succ, rows, fields = !gvc.disable, gvc.rows, gvc.fields
return
}
succ = false
return
}

// Disable disables the global variabe cache, used in test only.
func (gvc *GlobalVariableCache) Disable() {
gvc.Lock()
defer gvc.Unlock()
gvc.disable = true
return
}

// GetGlobalVarsCache gets the global variable cache.
func (do *Domain) GetGlobalVarsCache() *GlobalVariableCache {
return &do.gvc
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
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -49,7 +49,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20171020084629-8d44bfdf1030
github.com/pingcap/kvproto v0.0.0-20190429124202-32a5ba2af0f7
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596
github.com/pingcap/parser v0.0.0-20190620042621-a13211687e55
github.com/pingcap/parser v0.0.0-20190701060323-a2aa507d6352
github.com/pingcap/pd v2.1.12+incompatible
github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible
github.com/pingcap/tipb v0.0.0-20180910045846-371b48b15d93
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -103,8 +103,8 @@ github.com/pingcap/kvproto v0.0.0-20190429124202-32a5ba2af0f7 h1:+wEqJTc74Jvoxen
github.com/pingcap/kvproto v0.0.0-20190429124202-32a5ba2af0f7/go.mod h1:0gwbe1F2iBIjuQ9AH0DbQhL+Dpr5GofU8fgYyXk+ykk=
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596 h1:t2OQTpPJnrPDGlvA+3FwJptMTt6MEPdzK1Wt99oaefQ=
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596/go.mod h1:WpHUKhNZ18v116SvGrmjkA9CBhYmuUTKL+p8JC9ANEw=
github.com/pingcap/parser v0.0.0-20190620042621-a13211687e55 h1:JSr9saxYwDyIte70taWbKX83fCUww80HjrcyDnFokBg=
github.com/pingcap/parser v0.0.0-20190620042621-a13211687e55/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/parser v0.0.0-20190701060323-a2aa507d6352 h1:bqNncrTvLJyxLx8rHr5tDe4wYhfVFoegj+LrOoFwwuM=
github.com/pingcap/parser v0.0.0-20190701060323-a2aa507d6352/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v2.1.12+incompatible h1:6N3LBxx2aSZqT+IWEG730EDNDttP7dXO8J6yvBh+HXw=
github.com/pingcap/pd v2.1.12+incompatible/go.mod h1:nD3+EoYes4+aNNODO99ES59V83MZSI+dFbhyr667a0E=
github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible h1:e9Gi/LP9181HT3gBfSOeSBA+5JfemuE4aEAhqNgoE4k=
Expand Down
2 changes: 1 addition & 1 deletion infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -526,7 +526,7 @@ var tableProcesslistCols = []columnInfo{
{"ID", mysql.TypeLonglong, 21, mysql.NotNullFlag, 0, nil},
{"USER", mysql.TypeVarchar, 16, mysql.NotNullFlag, "", nil},
{"HOST", mysql.TypeVarchar, 64, mysql.NotNullFlag, "", nil},
{"DB", mysql.TypeVarchar, 64, mysql.NotNullFlag, "", nil},
{"DB", mysql.TypeVarchar, 64, 0, nil, nil},
{"COMMAND", mysql.TypeVarchar, 16, mysql.NotNullFlag, "", nil},
{"TIME", mysql.TypeLong, 7, mysql.NotNullFlag, 0, nil},
{"STATE", mysql.TypeVarchar, 7, 0, nil, nil},
Expand Down
46 changes: 46 additions & 0 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -334,6 +334,52 @@ func (s *testTableSuite) TestSomeTables(c *C) {
fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s 0", "do something"),
fmt.Sprintf("2 user-2 localhost test Init DB 9223372036 2 %s 0", strings.Repeat("x", 101)),
))

sm = &mockSessionManager{make(map[uint64]*util.ProcessInfo, 2)}
sm.processInfoMap[1] = &util.ProcessInfo{
ID: 1,
User: "user-1",
Host: "localhost",
DB: "information_schema",
Command: byte(1),
State: 1,
Info: nil,
StmtCtx: tk.Se.GetSessionVars().StmtCtx,
}
sm.processInfoMap[2] = &util.ProcessInfo{
ID: 2,
User: "user-2",
Host: "localhost",
DB: nil,
Command: byte(2),
State: 2,
Info: strings.Repeat("x", 101),
StmtCtx: tk.Se.GetSessionVars().StmtCtx,
}
tk.Se.SetSessionManager(sm)
tk.MustQuery("select * from information_schema.PROCESSLIST order by ID;").Check(
testkit.Rows(
fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s 0", "<nil>"),
fmt.Sprintf("2 user-2 localhost <nil> Init DB 9223372036 2 %s 0", strings.Repeat("x", 101)),
))
tk.MustQuery("SHOW PROCESSLIST;").Sort().Check(
testkit.Rows(
fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s 0", "<nil>"),
fmt.Sprintf("2 user-2 localhost <nil> Init DB 9223372036 2 %s 0", strings.Repeat("x", 100)),
))
tk.MustQuery("SHOW FULL PROCESSLIST;").Sort().Check(
testkit.Rows(
fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s 0", "<nil>"),
fmt.Sprintf("2 user-2 localhost <nil> Init DB 9223372036 2 %s 0", strings.Repeat("x", 101)),
))
tk.MustQuery("select * from information_schema.PROCESSLIST where db is null;").Check(
testkit.Rows(
fmt.Sprintf("2 user-2 localhost <nil> Init DB 9223372036 2 %s 0", strings.Repeat("x", 101)),
))
tk.MustQuery("select * from information_schema.PROCESSLIST where Info is null;").Check(
testkit.Rows(
fmt.Sprintf("1 user-1 localhost information_schema Quit 9223372036 1 %s 0", "<nil>"),
))
}

func (s *testSuite) TestSchemataCharacterSet(c *C) {
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))
}
Loading

0 comments on commit 28f12b5

Please sign in to comment.