diff --git a/executor/ddl_test.go b/executor/ddl_test.go index 7e1577c2e192c..9ea8a49d0fc4e 100644 --- a/executor/ddl_test.go +++ b/executor/ddl_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/domain" + "github.com/pingcap/tidb/meta/autoid" plannercore "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/table" @@ -406,6 +407,24 @@ func (s *testSuite) TestShardRowIDBits(c *C) { _, err = tk.Exec("alter table auto shard_row_id_bits = 4") c.Assert(err, NotNil) tk.MustExec("alter table auto shard_row_id_bits = 0") + + // 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")) + c.Assert(err, IsNil) + maxID := 1<<(64-15-1) - 1 + err = tbl.RebaseAutoID(tk.Se, int64(maxID)-1, false) + c.Assert(err, IsNil) + tk.MustExec("insert into t1 values(1)") + + // continue inserting will fail. + _, err = tk.Exec("insert into t1 values(2)") + c.Assert(autoid.ErrAutoincReadFailed.Equal(err), IsTrue, Commentf("err:%v", err)) + _, err = tk.Exec("insert into t1 values(3)") + c.Assert(autoid.ErrAutoincReadFailed.Equal(err), IsTrue, Commentf("err:%v", err)) } func (s *testSuite) TestMaxHandleAddIndex(c *C) { diff --git a/meta/autoid/errors.go b/meta/autoid/errors.go new file mode 100644 index 0000000000000..44ef83650a202 --- /dev/null +++ b/meta/autoid/errors.go @@ -0,0 +1,32 @@ +// Copyright 2018 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid + +import ( + "github.com/pingcap/parser/mysql" + "github.com/pingcap/parser/terror" +) + +// Error instances. +var ( + ErrAutoincReadFailed = terror.ClassAutoid.New(mysql.ErrAutoincReadFailed, mysql.MySQLErrName[mysql.ErrAutoincReadFailed]) +) + +func init() { + // Map error codes to mysql error codes. + tableMySQLErrCodes := map[terror.ErrCode]uint16{ + mysql.ErrAutoincReadFailed: mysql.ErrAutoincReadFailed, + } + terror.ErrClassToMySQLCodes[terror.ClassAutoid] = tableMySQLErrCodes +} diff --git a/table/tables/tables.go b/table/tables/tables.go index 8644f3cb1a06a..15891322fadb9 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -909,6 +909,16 @@ func (t *tableCommon) AllocAutoID(ctx sessionctx.Context) (int64, error) { return 0, errors.Trace(err) } if t.meta.ShardRowIDBits > 0 { + if t.overflowShardBits(rowID) { + // If overflow, the rowID may be duplicated. For examples, + // t.meta.ShardRowIDBits = 4 + // rowID = 0010111111111111111111111111111111111111111111111111111111111111 + // shard = 01000000000000000000000000000000000000000000000000000000000000000 + // will be duplicated with: + // rowID = 0100111111111111111111111111111111111111111111111111111111111111 + // shard = 0010000000000000000000000000000000000000000000000000000000000000 + return 0, autoid.ErrAutoincReadFailed + } txnCtx := ctx.GetSessionVars().TxnCtx if txnCtx.Shard == nil { shard := t.calcShard(txnCtx.StartTS) @@ -919,6 +929,12 @@ func (t *tableCommon) AllocAutoID(ctx sessionctx.Context) (int64, error) { return rowID, nil } +// overflowShardBits check whether the rowID overflow `1<<(64-t.meta.ShardRowIDBits-1) -1`. +func (t *tableCommon) overflowShardBits(rowID int64) bool { + mask := (1< 0 +} + func (t *tableCommon) calcShard(startTS uint64) int64 { var buf [8]byte binary.LittleEndian.PutUint64(buf[:], startTS)