Skip to content

Commit

Permalink
ddl: Add some limit for auto_random (#17119) (#17248)
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored May 20, 2020
1 parent 4211cb7 commit 0414aa5
Show file tree
Hide file tree
Showing 7 changed files with 95 additions and 77 deletions.
12 changes: 7 additions & 5 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1104,6 +1104,10 @@ func setTableAutoRandomBits(tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) e
if !allowAutoRandom {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomExperimentalDisabledErrMsg)
}
if col.Tp.Tp != mysql.TypeLonglong {
return ErrInvalidAutoRandom.GenWithStackByArgs(
fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(col.Tp.Tp)))
}
if !tbInfo.PKIsHandle || col.Name.Name.L != pkColName.L {
errMsg := fmt.Sprintf(autoid.AutoRandomPKisNotHandleErrMsg, col.Name.Name.O)
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
Expand All @@ -1119,13 +1123,11 @@ func setTableAutoRandomBits(tbInfo *model.TableInfo, colDefs []*ast.ColumnDef) e
if err != nil {
return errors.Trace(err)
}

layout := autoid.NewAutoRandomIDLayout(col.Tp, autoRandBits)
if autoRandBits == 0 {
return ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomNonPositive)
} else if autoRandBits >= layout.TypeBitsLength {
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, col.Name.Name.L,
layout.TypeBitsLength, autoRandBits, col.Name.Name.L, layout.TypeBitsLength-1)
} else if autoRandBits > autoid.MaxAutoRandomBits {
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg,
autoid.MaxAutoRandomBits, autoRandBits, col.Name.Name.O)
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
tbInfo.AutoRandomBits = autoRandBits
Expand Down
80 changes: 42 additions & 38 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -849,18 +849,23 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
assertWithAutoInc := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomIncompatibleWithAutoIncErrMsg)
}
assertOverflow := func(sql, colType string, autoRandBits, maxFieldLength uint64) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomOverflowErrMsg, colType, maxFieldLength, autoRandBits, colType, maxFieldLength-1)
assertOverflow := func(sql, colName string, autoRandBits uint64) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomOverflowErrMsg, autoid.MaxAutoRandomBits, autoRandBits, colName)
}
assertModifyColType := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomModifyColTypeErrMsg)
_, err := tk.Exec(sql)
c.Assert(err, NotNil)
c.Assert(strings.Contains(err.Error(), "[ddl:203]unsupported modify column length"), IsTrue)
}
assertDefault := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomIncompatibleWithDefaultValueErrMsg)
}
assertNonPositive := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomNonPositive)
}
assertBigIntOnly := func(sql, colType string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomOnNonBigIntColumn, colType)
}
mustExecAndDrop := func(sql string, fns ...func()) {
tk.MustExec(sql)
for _, f := range fns {
Expand All @@ -871,67 +876,63 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {

testutil.ConfigTestUtils.SetupAutoRandomTestConfig()
defer testutil.ConfigTestUtils.RestoreAutoRandomTestConfig()

// Only bigint column can set auto_random
assertBigIntOnly("create table t (a char primary key auto_random(3), b int)", "char")
assertBigIntOnly("create table t (a varchar(255) primary key auto_random(3), b int)", "varchar")
assertBigIntOnly("create table t (a timestamp primary key auto_random(3), b int)", "timestamp")

// PKIsHandle, but auto_random is defined on non-primary key.
assertPKIsNotHandle("create table t (a bigint auto_random (3) primary key, b int auto_random (3))", "b")
assertPKIsNotHandle("create table t (a bigint auto_random (3), b int auto_random(3), primary key(a))", "b")
assertPKIsNotHandle("create table t (a bigint auto_random (3), b int auto_random(3) primary key)", "a")
assertPKIsNotHandle("create table t (a bigint auto_random (3) primary key, b bigint auto_random (3))", "b")
assertPKIsNotHandle("create table t (a bigint auto_random (3), b bigint auto_random(3), primary key(a))", "b")
assertPKIsNotHandle("create table t (a bigint auto_random (3), b bigint auto_random(3) primary key)", "a")

// PKIsNotHandle: no primary key.
assertPKIsNotHandle("create table t (a int auto_random(3), b int)", "a")
assertPKIsNotHandle("create table t (a bigint auto_random(3), b int)", "a")
// PKIsNotHandle: primary key is not integer column.
assertPKIsNotHandle("create table t (a char primary key auto_random(3), b int)", "a")
assertPKIsNotHandle("create table t (a varchar(255) primary key auto_random(3), b int)", "a")
assertPKIsNotHandle("create table t (a timestamp primary key auto_random(3), b int)", "a")
// PKIsNotHandle: primary key is not a single column.
assertPKIsNotHandle("create table t (a bigint auto_random(3), b int, primary key (a, b))", "a")
assertPKIsNotHandle("create table t (a int auto_random(3), b int, c char, primary key (a, c))", "a")
assertPKIsNotHandle("create table t (a bigint auto_random(3), b bigint, primary key (a, b))", "a")
assertPKIsNotHandle("create table t (a bigint auto_random(3), b int, c char, primary key (a, c))", "a")

// Can not set auto_random along with auto_increment.
assertWithAutoInc("create table t (a bigint auto_random(3) primary key auto_increment)")
assertWithAutoInc("create table t (a bigint primary key auto_increment auto_random(3))")
assertWithAutoInc("create table t (a bigint auto_increment primary key auto_random(3))")
assertWithAutoInc("create table t (a bigint auto_random(3) auto_increment, primary key (a))")

// Overflow data type max length.
assertOverflow("create table t (a bigint auto_random(65) primary key)", "a", 65, 64)
assertOverflow("create table t (a int auto_random(33) primary key)", "a", 33, 32)
assertOverflow("create table t (a mediumint auto_random(25) primary key)", "a", 25, 24)
assertOverflow("create table t (a smallint auto_random(17) primary key)", "a", 17, 16)
assertOverflow("create table t (a tinyint auto_random(9) primary key)", "a", 9, 8)

assertNonPositive("create table t (a bigint auto_random(0) primary key)")

// Can not set auto_random along with default.
assertDefault("create table t (a int auto_random primary key default 3)")
assertDefault("create table t (a bigint auto_random primary key default 3)")
assertDefault("create table t (a bigint auto_random(2) primary key default 5)")
mustExecAndDrop("create table t (a int auto_random primary key)", func() {
assertDefault("alter table t modify column a int auto_random default 3")
mustExecAndDrop("create table t (a bigint auto_random primary key)", func() {
assertDefault("alter table t modify column a bigint auto_random default 3")
})

// Basic usage.
mustExecAndDrop("create table t (a bigint auto_random(4) primary key, b varchar(255))")
mustExecAndDrop("create table t (a bigint primary key auto_random(4), b varchar(255))")
mustExecAndDrop("create table t (a bigint auto_random(4), b varchar(255), primary key (a))")
// Overflow data type max length.
assertOverflow("create table t (a bigint auto_random(64) primary key)", "a", 64)
assertOverflow("create table t (a bigint auto_random(16) primary key)", "a", 16)

// Different primary key field types.
assertNonPositive("create table t (a bigint auto_random(0) primary key)")
_, err := tk.Exec("create table t (a bigint auto_random(-1) primary key)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, `[parser:1064]You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 38 near "-1) primary key)" `)

// Basic usage.
mustExecAndDrop("create table t (a bigint auto_random(1) primary key)")
mustExecAndDrop("create table t (a bigint auto_random(4) primary key)")
mustExecAndDrop("create table t (a int auto_random(4) primary key)")
mustExecAndDrop("create table t (a mediumint auto_random(4) primary key)")
mustExecAndDrop("create table t (a smallint auto_random(4) primary key)")
mustExecAndDrop("create table t (a tinyint auto_random(4) primary key)")
mustExecAndDrop("create table t (a bigint auto_random(15) primary key)")
mustExecAndDrop("create table t (a bigint primary key auto_random(4))")
mustExecAndDrop("create table t (a bigint auto_random(4), primary key (a))")

// Auto_random can occur multiple times like other column attributes.
mustExecAndDrop("create table t (a bigint auto_random(3) auto_random(2) primary key)")
mustExecAndDrop("create table t (a int, b bigint auto_random(3) primary key auto_random(2))")
mustExecAndDrop("create table t (a int auto_random(1) auto_random(2) auto_random(3), primary key (a))")
mustExecAndDrop("create table t (a bigint, b bigint auto_random(3) primary key auto_random(2))")
mustExecAndDrop("create table t (a bigint auto_random(1) auto_random(2) auto_random(3), primary key (a))")

// Add/drop the auto_random attribute is not allowed.
mustExecAndDrop("create table t (a bigint auto_random(3) primary key)", func() {
assertAlterValue("alter table t modify column a bigint")
assertAlterValue("alter table t change column a b bigint")
})
mustExecAndDrop("create table t (a int, b char, c int auto_random(3), primary key(c))", func() {
mustExecAndDrop("create table t (a bigint, b char, c bigint auto_random(3), primary key(c))", func() {
assertAlterValue("alter table t modify column c bigint")
assertAlterValue("alter table t change column c d bigint")
})
Expand All @@ -941,7 +942,10 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
})

// Modifying the field type of a auto_random column is not allowed.
mustExecAndDrop("create table t (a tinyint primary key auto_random(3))", func() {
// Here the throw error is `ERROR 8200 (HY000): Unsupported modify column: length 11 is less than origin 20`,
// instead of `ERROR 8216 (HY000): Invalid auto random: modifying the auto_random column type is not supported`
// Because the origin column is `bigint`, it can not change to any other column type in TiDB limitation.
mustExecAndDrop("create table t (a bigint primary key auto_random(3))", func() {
assertModifyColType("alter table t modify column a int auto_random(3)")
assertModifyColType("alter table t modify column a mediumint auto_random(3)")
assertModifyColType("alter table t modify column a smallint auto_random(3)")
Expand Down
27 changes: 10 additions & 17 deletions executor/ddl_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -871,36 +871,29 @@ func (s *testAutoRandomSuite) TestAutoRandomBitsData(c *C) {
}

// Test explicit insert.
tk.MustExec("create table t (a tinyint primary key auto_random(2), b int)")
for i := 1; i <= 100; i++ {
tk.MustExec("insert into t values (?, ?)", i, i)
autoRandBitsUpperBound := 2<<47 - 1
tk.MustExec("create table t (a bigint primary key auto_random(15), b int)")
for i := -10; i < 10; i++ {
tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i+autoRandBitsUpperBound, i))
}
_, err = tk.Exec("insert into t (b) values (0)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error())
tk.MustExec("drop table t")

// Test overflow.
tk.MustExec("create table t (a tinyint primary key auto_random(2), b int)")
for i := 0; i < 31; /* 2^(8-2-1)-1 */ i++ {
tk.MustExec(fmt.Sprintf("insert into t (b) values (%d)", i))
}
_, err = tk.Exec("insert into t (b) values (0)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error())
tk.MustExec("drop table t")

// Test rebase.
tk.MustExec("create table t (a tinyint primary key auto_random(2), b int)")
tk.MustExec("insert into t values (31, 2)")
tk.MustExec("create table t (a bigint primary key auto_random(15), b int)")
// Here we cannot fill the all values for a `bigint` column,
// so firstly we rebase auto_rand to the position before overflow.
tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", autoRandBitsUpperBound, 1))
_, err = tk.Exec("insert into t (b) values (0)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error())
tk.MustExec("drop table t")

tk.MustExec("create table t (a tinyint primary key auto_random(2), b int)")
tk.MustExec("create table t (a bigint primary key auto_random(15), b int)")
tk.MustExec("insert into t values (1, 2)")
tk.MustExec("update t set a = 31 where a = 1")
tk.MustExec(fmt.Sprintf("update t set a = %d where a = 1", autoRandBitsUpperBound))
_, err = tk.Exec("insert into t (b) values (0)")
c.Assert(err, NotNil)
c.Assert(err.Error(), Equals, autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error())
Expand Down
4 changes: 2 additions & 2 deletions executor/seqtest/seq_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1088,7 +1088,7 @@ func (s *seqTestSuite) TestAutoRandIDRetry(c *C) {
tk.MustExec("create database if not exists auto_random_retry")
tk.MustExec("use auto_random_retry")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t (id int auto_random(3) primary key)")
tk.MustExec("create table t (id bigint auto_random(3) primary key)")

extractMaskedOrderedHandles := func() []int64 {
handles, err := ddltestutil.ExtractAllTableHandles(tk.Se, "auto_random_retry", "t")
Expand Down Expand Up @@ -1166,7 +1166,7 @@ func (s *seqTestSuite) TestAutoRandRecoverTable(c *C) {
defer autoid.SetStep(stp)

// Check rebase auto_random id.
tk.MustExec("create table t_recover_auto_rand (a int auto_random(5) primary key);")
tk.MustExec("create table t_recover_auto_rand (a bigint auto_random(5) primary key);")
tk.MustExec("insert into t_recover_auto_rand values (),(),()")
tk.MustExec("drop table t_recover_auto_rand")
tk.MustExec("recover table t_recover_auto_rand")
Expand Down
8 changes: 6 additions & 2 deletions executor/write.go
Original file line number Diff line number Diff line change
Expand Up @@ -201,8 +201,12 @@ func rebaseAutoRandomValue(sctx sessionctx.Context, t table.Table, newData *type
if err != nil {
return err
}
shardBits := tableInfo.AutoRandomBits + 1 // sign bit is reserved.
recordID = recordID << shardBits >> shardBits
if recordID < 0 {
return nil
}
layout := autoid.NewAutoRandomIDLayout(&col.FieldType, tableInfo.AutoRandomBits)
// Set bits except incremental_bits to zero.
recordID = recordID & (1<<layout.IncrementalBits - 1)
return t.Allocator(sctx, autoid.AutoRandomType).Rebase(tableInfo.ID, recordID, true)
}

Expand Down
35 changes: 23 additions & 12 deletions meta/autoid/autoid.go
Original file line number Diff line number Diff line change
Expand Up @@ -25,10 +25,10 @@ import (
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/types"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta"
"github.com/pingcap/tidb/metrics"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)
Expand All @@ -47,6 +47,9 @@ const RowIDBitLength = 64
// DefaultAutoRandomBits is the default value of auto sharding.
const DefaultAutoRandomBits = 5

// MaxAutoRandomBits is the max value of auto sharding.
const MaxAutoRandomBits = 15

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

Expand Down Expand Up @@ -567,8 +570,14 @@ func generateAutoIDByAllocType(m *meta.Meta, dbID, tableID, step int64, allocTyp
}

// AutoRandomIDLayout is used to calculate the bits length of different section in auto_random id.
// Layout(64 bits):
// [zero_padding] [sign_bit] [shard_bits] [incremental_bits]
// The primary key with auto_random can only be `bigint` column, the total layout length of auto random is 64 bits.
// These are two type of layout:
// 1. Signed bigint:
// | [sign_bit] | [shard_bits] | [incremental_bits] |
// sign_bit(1 fixed) + shard_bits(15 max) + incremental_bits(the rest) = total_layout_bits(64 fixed)
// 2. Unsigned bigint:
// | [shard_bits] | [incremental_bits] |
// shard_bits(15 max) + incremental_bits(the rest) = total_layout_bits(64 fixed)
// Please always use NewAutoRandomIDLayout() to instantiate.
type AutoRandomIDLayout struct {
FieldType *types.FieldType
Expand All @@ -581,17 +590,19 @@ type AutoRandomIDLayout struct {

// NewAutoRandomIDLayout create an instance of AutoRandomIDLayout.
func NewAutoRandomIDLayout(fieldType *types.FieldType, shardBits uint64) *AutoRandomIDLayout {
layout := &AutoRandomIDLayout{
FieldType: fieldType,
ShardBits: shardBits,
typeBitsLength := uint64(mysql.DefaultLengthOfMysqlTypes[mysql.TypeLonglong] * 8)
incrementalBits := typeBitsLength - shardBits
hasSignBit := !mysql.HasUnsignedFlag(fieldType.Flag)
if hasSignBit {
incrementalBits -= 1
}
layout.TypeBitsLength = uint64(mysql.DefaultLengthOfMysqlTypes[fieldType.Tp] * 8)
layout.HasSignBit = !mysql.HasUnsignedFlag(fieldType.Flag)
layout.IncrementalBits = layout.TypeBitsLength - shardBits
if layout.HasSignBit {
layout.IncrementalBits -= 1
return &AutoRandomIDLayout{
FieldType: fieldType,
ShardBits: shardBits,
TypeBitsLength: typeBitsLength,
IncrementalBits: incrementalBits,
HasSignBit: hasSignBit,
}
return layout
}

// IncrementalBitsCapacity returns the max capacity of incremental section of the current layout.
Expand Down
6 changes: 5 additions & 1 deletion meta/autoid/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -51,11 +51,15 @@ const (
// AutoRandomIncompatibleWithDefaultValueErrMsg is reported when auto_random and default are specified on the same column.
AutoRandomIncompatibleWithDefaultValueErrMsg = "auto_random is incompatible with default"
// AutoRandomOverflowErrMsg is reported when auto_random is greater than max length of a MySQL data type.
AutoRandomOverflowErrMsg = "Bits of column `%s` is %d, but auto_random bits is %d. Max allowed auto_random bits for column `%s` is %d"
AutoRandomOverflowErrMsg = "max allowed auto_random bits is %d, but got %d on column `%s`"
// AutoRandomModifyColTypeErrMsg is reported when a user is trying to modify the type of a column specified with auto_random.
AutoRandomModifyColTypeErrMsg = "modifying the auto_random column type is not supported"
// AutoRandomAlterErrMsg is reported when a user is trying to add/drop/modify the value of auto_random attribute.
AutoRandomAlterErrMsg = "adding/dropping/modifying auto_random is not supported"
// AutoRandomNonPositive is reported then a user specifies a non-positive value for auto_random.
AutoRandomNonPositive = "the value of auto_random should be positive"
// AutoRandomAvailableAllocTimesNote is reported when a table containing auto_random is created.
AutoRandomAvailableAllocTimesNote = "Available implicit allocation times: %d"
// AutoRandomOnNonBigIntColumn is reported when define auto random to non bigint column
AutoRandomOnNonBigIntColumn = "auto_random option must be defined on `bigint` column, but not on `%s` column"
)

0 comments on commit 0414aa5

Please sign in to comment.