Skip to content

Commit

Permalink
ddl: fix alter table share rowid bit problem (pingcap#9868)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 committed May 13, 2019
1 parent 7ad9b1a commit c6a90dc
Show file tree
Hide file tree
Showing 8 changed files with 100 additions and 11 deletions.
37 changes: 37 additions & 0 deletions ddl/db_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -4631,3 +4631,40 @@ func (s *testDBSuite) TestCanceledJobTakeTime(c *C) {
hook = &ddl.TestDDLCallback{}
s.dom.DDL().(ddl.DDLForTest).SetHook(hook)
}

func (s *testDBSuite) TestAlterShardRowIDBits(c *C) {
s.tk = testkit.NewTestKit(c, s.store)
tk := s.tk

tk.MustExec("use test")
// Test alter shard_row_id_bits
tk.MustExec("drop table if exists t1")
defer tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a int) shard_row_id_bits = 5")
tk.MustExec(fmt.Sprintf("alter table t1 auto_increment = %d;", 1<<56))
tk.MustExec("insert into t1 set a=1;")

// Test increase shard_row_id_bits failed by overflow global auto ID.
_, err := tk.Exec("alter table t1 SHARD_ROW_ID_BITS = 10;")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[autoid:1467]shard_row_id_bits 10 will cause next global auto ID overflow")

// Test reduce shard_row_id_bits will be ok.
tk.MustExec("alter table t1 SHARD_ROW_ID_BITS = 3;")
checkShardRowID := func(maxShardRowIDBits, shardRowIDBits uint64) {
tbl := testGetTableByName(c, tk.Se, "test", "t1")
c.Assert(tbl.Meta().MaxShardRowIDBits == maxShardRowIDBits, IsTrue)
c.Assert(tbl.Meta().ShardRowIDBits == shardRowIDBits, IsTrue)
}
checkShardRowID(5, 3)

// Test reduce shard_row_id_bits but calculate overflow should use the max record shard_row_id_bits.
tk.MustExec("drop table if exists t1")
tk.MustExec("create table t1 (a int) shard_row_id_bits = 10")
tk.MustExec("alter table t1 SHARD_ROW_ID_BITS = 5;")
checkShardRowID(10, 5)
tk.MustExec(fmt.Sprintf("alter table t1 auto_increment = %d;", 1<<56))
_, err = tk.Exec("insert into t1 set a=1;")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, "[autoid:1467]Failed to read auto-increment value from storage engine")
}
6 changes: 5 additions & 1 deletion ddl/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,7 @@ type ddl struct {

*ddlCtx
workers map[workerType]*worker
sessPool *sessionPool
}

// ddlCtx is the context when we use worker to handle DDL jobs.
Expand Down Expand Up @@ -374,7 +375,7 @@ func (d *ddl) start(ctx context.Context, ctxPool *pools.ResourcePool) {
if RunWorker {
err := d.ownerManager.CampaignOwner(ctx)
terror.Log(errors.Trace(err))

d.sessPool = &sessionPool{resPool: ctxPool}
d.workers = make(map[workerType]*worker, 2)
d.workers[generalWorker] = newWorker(generalWorker, d.store, ctxPool)
d.workers[addIdxWorker] = newWorker(addIdxWorker, d.store, ctxPool)
Expand Down Expand Up @@ -414,6 +415,9 @@ func (d *ddl) close() {
for _, worker := range d.workers {
worker.close()
}
if d.sessPool != nil {
d.sessPool.close()
}
logutil.Logger(ddlLogCtx).Info("[ddl] closing DDL", zap.String("ID", d.uuid), zap.Duration("takeTime", time.Since(startTime)))
}

Expand Down
9 changes: 9 additions & 0 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1166,6 +1166,7 @@ func handleTableOptions(options []*ast.TableOption, tbInfo *model.TableInfo) err
if tbInfo.ShardRowIDBits > shardRowIDBitsMax {
tbInfo.ShardRowIDBits = shardRowIDBitsMax
}
tbInfo.MaxShardRowIDBits = tbInfo.ShardRowIDBits
}
}

Expand Down Expand Up @@ -1356,6 +1357,14 @@ func (d *ddl) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint
if hasAutoIncrementColumn(t.Meta()) && uVal != 0 {
return errUnsupportedShardRowIDBits
}
if uVal == t.Meta().ShardRowIDBits {
// Nothing need to do.
return nil
}
err = verifyNoOverflowShardBits(d.sessPool, t, uVal)
if err != nil {
return err
}
job := &model.Job{
Type: model.ActionShardRowID,
SchemaID: schema.ID,
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -510,7 +510,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
case model.ActionRenameTable:
ver, err = onRenameTable(t, job)
case model.ActionShardRowID:
ver, err = onShardRowID(t, job)
ver, err = w.onShardRowID(d, t, job)
case model.ActionModifyTableComment:
ver, err = onModifyTableComment(t, job)
case model.ActionAddTablePartition:
Expand Down
38 changes: 36 additions & 2 deletions ddl/table.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/table/tables"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
Expand Down Expand Up @@ -330,7 +331,7 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job) (ver int64,
return ver, nil
}

func onShardRowID(t *meta.Meta, job *model.Job) (ver int64, _ error) {
func (w *worker) onShardRowID(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) {
var shardRowIDBits uint64
err := job.DecodeArgs(&shardRowIDBits)
if err != nil {
Expand All @@ -342,7 +343,22 @@ func onShardRowID(t *meta.Meta, job *model.Job) (ver int64, _ error) {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}
tblInfo.ShardRowIDBits = shardRowIDBits
if shardRowIDBits < tblInfo.ShardRowIDBits {
tblInfo.ShardRowIDBits = shardRowIDBits
} else {
tbl, err := getTable(d.store, job.SchemaID, tblInfo)
if err != nil {
return ver, errors.Trace(err)
}
err = verifyNoOverflowShardBits(w.sessPool, tbl, shardRowIDBits)
if err != nil {
job.State = model.JobStateCancelled
return ver, err
}
tblInfo.ShardRowIDBits = shardRowIDBits
// MaxShardRowIDBits use to check the overflow of auto ID.
tblInfo.MaxShardRowIDBits = shardRowIDBits
}
ver, err = updateVersionAndTableInfo(t, job, tblInfo, true)
if err != nil {
job.State = model.JobStateCancelled
Expand All @@ -352,6 +368,24 @@ func onShardRowID(t *meta.Meta, job *model.Job) (ver int64, _ error) {
return ver, nil
}

func verifyNoOverflowShardBits(s *sessionPool, tbl table.Table, shardRowIDBits uint64) error {
ctx, err := s.get()
if err != nil {
return errors.Trace(err)
}
defer s.put(ctx)

// Check next global max auto ID first.
autoIncID, err := tbl.Allocator(ctx).NextGlobalAutoID(tbl.Meta().ID)
if err != nil {
return errors.Trace(err)
}
if tables.OverflowShardBits(autoIncID, shardRowIDBits) {
return autoid.ErrAutoincReadFailed.GenWithStack("shard_row_id_bits %d will cause next global auto ID overflow", shardRowIDBits)
}
return nil
}

func onRenameTable(t *meta.Meta, job *model.Job) (ver int64, _ error) {
var oldSchemaID int64
var tableName model.CIStr
Expand Down
4 changes: 3 additions & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,7 @@ require (
github.com/pingcap/goleveldb v0.0.0-20171020084629-8d44bfdf1030
github.com/pingcap/kvproto v0.0.0-20190226063853-f6c0b7ffff11
github.com/pingcap/log v0.0.0-20190307075452-bd41d9273596
github.com/pingcap/parser v0.0.0-20190505094039-595d728571a7
github.com/pingcap/parser v0.0.0-20190409044748-a0b301443a30
github.com/pingcap/pd v2.1.0-rc.4+incompatible
github.com/pingcap/tidb-tools v2.1.3-0.20190116051332-34c808eef588+incompatible
github.com/pingcap/tipb v0.0.0-20180910045846-371b48b15d93
Expand Down Expand Up @@ -78,3 +78,5 @@ require (
gopkg.in/natefinch/lumberjack.v2 v2.0.0
gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 // indirect
)

replace github.com/pingcap/parser => github.com/crazycs520/parser v0.0.0-20190513122559-d0129540cb77
6 changes: 4 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@ github.com/coreos/go-systemd v0.0.0-20180202092358-40e2722dffea h1:IHPWgevPcOUjT
github.com/coreos/go-systemd v0.0.0-20180202092358-40e2722dffea/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4=
github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf h1:CAKfRE2YtTUIjjh1bkBtyYFaUT/WmOqsJjgtihT0vMI=
github.com/coreos/pkg v0.0.0-20160727233714-3ac0863d7acf/go.mod h1:E3G3o1h8I7cfcXa63jLwjI0eiQQMgzzUDFVpN/nH/eA=
github.com/crazycs520/parser v0.0.0-20190513122559-d0129540cb77 h1:N/ylJUgltydY0TXg2TMtsCqkaR6AWv0uhY/RfCgwBKg=
github.com/crazycs520/parser v0.0.0-20190513122559-d0129540cb77/go.mod h1:xLjI+gnWYexq011WPMEvCNS8rFM9qe1vdojIEzSKPuc=
github.com/cznic/mathutil v0.0.0-20160613104831-78ad7f262603 h1:hhR9hTi0ligs11JjfGDBP332clNOJRdW0Ci5oHtEC+0=
github.com/cznic/mathutil v0.0.0-20160613104831-78ad7f262603/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM=
github.com/cznic/sortutil v0.0.0-20150617083342-4c7342852e65 h1:hxuZop6tSoOi0sxFzoGGYdRqNrPubyaIf9KoBG9tPiE=
Expand Down Expand Up @@ -101,8 +103,8 @@ github.com/pingcap/kvproto v0.0.0-20190226063853-f6c0b7ffff11 h1:e81flSfRbbMW5RU
github.com/pingcap/kvproto v0.0.0-20190226063853-f6c0b7ffff11/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-20190505094039-595d728571a7 h1:cbTQGLE0X69qL2nrvtG9HP4u5sBdVGyoIJOhc+KtJXc=
github.com/pingcap/parser v0.0.0-20190505094039-595d728571a7/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/parser v0.0.0-20190409044748-a0b301443a30 h1:Cu+VJBHLUqI0TFj/0Kya4L1iHIJZ3VbtZcEwv+3zOxQ=
github.com/pingcap/parser v0.0.0-20190409044748-a0b301443a30/go.mod h1:1FNvfp9+J0wvc4kl8eGNh7Rqrxveg15jJoWo/a0uHwA=
github.com/pingcap/pd v2.1.0-rc.4+incompatible h1:/buwGk04aHO5odk/+O8ZOXGs4qkUjYTJ2UpCJXna8NE=
github.com/pingcap/pd v2.1.0-rc.4+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
9 changes: 5 additions & 4 deletions table/tables/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -910,7 +910,8 @@ func (t *tableCommon) AllocAutoID(ctx sessionctx.Context) (int64, error) {
return 0, errors.Trace(err)
}
if t.meta.ShardRowIDBits > 0 {
if t.overflowShardBits(rowID) {
// Use max record ShardRowIDBits to check overflow.
if OverflowShardBits(rowID, t.meta.MaxShardRowIDBits) {
// If overflow, the rowID may be duplicated. For examples,
// t.meta.ShardRowIDBits = 4
// rowID = 0010111111111111111111111111111111111111111111111111111111111111
Expand All @@ -930,9 +931,9 @@ 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<<t.meta.ShardRowIDBits - 1) << (64 - t.meta.ShardRowIDBits - 1)
// OverflowShardBits checks whether the rowID overflow `1<<(64-shardRowIDBits-1) -1`.
func OverflowShardBits(rowID int64, shardRowIDBits uint64) bool {
mask := (1<<shardRowIDBits - 1) << (64 - shardRowIDBits - 1)
return rowID&int64(mask) > 0
}

Expand Down

0 comments on commit c6a90dc

Please sign in to comment.