Skip to content

Commit

Permalink
ddl: support change from auto_inc to auto_random through 'alter table' (
Browse files Browse the repository at this point in the history
  • Loading branch information
ti-srebot committed May 13, 2021
1 parent c7bd1d0 commit 9b2443a
Show file tree
Hide file tree
Showing 12 changed files with 281 additions and 124 deletions.
82 changes: 62 additions & 20 deletions ddl/column.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,6 @@ import (
"sync/atomic"
"time"

"github.com/cznic/mathutil"
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/ast"
Expand Down Expand Up @@ -353,7 +352,7 @@ func onSetDefaultValue(t *meta.Meta, job *model.Job) (ver int64, _ error) {
return updateColumnDefaultValue(t, job, newCol, &newCol.Name)
}

func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ error) {
func (w *worker) onModifyColumn(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) {
newCol := &model.ColumnInfo{}
oldColName := &model.CIStr{}
pos := &ast.ColumnPosition{}
Expand All @@ -365,12 +364,12 @@ func (w *worker) onModifyColumn(t *meta.Meta, job *model.Job) (ver int64, _ erro
return ver, errors.Trace(err)
}

return w.doModifyColumn(t, job, newCol, oldColName, pos, modifyColumnTp, updatedAutoRandomBits)
return w.doModifyColumn(d, t, job, newCol, oldColName, pos, modifyColumnTp, updatedAutoRandomBits)
}

// doModifyColumn updates the column information and reorders all columns.
func (w *worker) doModifyColumn(
t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr,
d *ddlCtx, t *meta.Meta, job *model.Job, newCol *model.ColumnInfo, oldName *model.CIStr,
pos *ast.ColumnPosition, modifyColumnTp byte, newAutoRandBits uint64) (ver int64, _ error) {
dbInfo, err := checkSchemaExistAndCancelNotExistJob(t, job)
if err != nil {
Expand Down Expand Up @@ -413,11 +412,12 @@ func (w *worker) doModifyColumn(
}
})

if newAutoRandBits > 0 {
if err := checkAndApplyNewAutoRandomBits(job, t, tblInfo, newCol, oldName, newAutoRandBits); err != nil {
return ver, errors.Trace(err)
}
err = checkAndApplyAutoRandomBits(d, t, dbInfo, tblInfo, oldCol, newCol, newAutoRandBits)
if err != nil {
job.State = model.JobStateCancelled
return ver, errors.Trace(err)
}

// Column from null to not null.
if !mysql.HasNotNullFlag(oldCol.Flag) && mysql.HasNotNullFlag(newCol.Flag) {
noPreventNullFlag := !mysql.HasPreventNullInsertFlag(oldCol.Flag)
Expand Down Expand Up @@ -508,31 +508,73 @@ func (w *worker) doModifyColumn(
return ver, nil
}

func checkAndApplyNewAutoRandomBits(job *model.Job, t *meta.Meta, tblInfo *model.TableInfo,
newCol *model.ColumnInfo, oldName *model.CIStr, newAutoRandBits uint64) error {
schemaID := job.SchemaID
func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, tblInfo *model.TableInfo,
oldCol *model.ColumnInfo, newCol *model.ColumnInfo, newAutoRandBits uint64) error {
if newAutoRandBits == 0 {
return nil
}
err := checkNewAutoRandomBits(m, dbInfo.ID, tblInfo.ID, oldCol, newCol, newAutoRandBits)
if err != nil {
return err
}
return applyNewAutoRandomBits(d, m, dbInfo, tblInfo, oldCol, newAutoRandBits)
}

// checkNewAutoRandomBits checks whether the new auto_random bits number can cause overflow.
func checkNewAutoRandomBits(m *meta.Meta, schemaID, tblID int64,
oldCol *model.ColumnInfo, newCol *model.ColumnInfo, newAutoRandBits uint64) error {
newLayout := autoid.NewShardIDLayout(&newCol.FieldType, newAutoRandBits)

// GenAutoRandomID first to prevent concurrent update.
_, err := t.GenAutoRandomID(schemaID, tblInfo.ID, 1)
allocTp := autoid.AutoRandomType
convertedFromAutoInc := mysql.HasAutoIncrementFlag(oldCol.Flag)
if convertedFromAutoInc {
allocTp = autoid.AutoIncrementType
}
// GenerateAutoID first to prevent concurrent update in DML.
_, err := autoid.GenerateAutoID(m, schemaID, tblID, 1, allocTp)
if err != nil {
return err
}
currentIncBitsVal, err := t.GetAutoRandomID(schemaID, tblInfo.ID)
currentIncBitsVal, err := autoid.GetAutoID(m, schemaID, tblID, allocTp)
if err != nil {
return err
}
// Find the max number of available shard bits by
// counting leading zeros in current inc part of auto_random ID.
availableBits := bits.LeadingZeros64(uint64(currentIncBitsVal))
isOccupyingIncBits := newLayout.TypeBitsLength-newLayout.IncrementalBits > uint64(availableBits)
if isOccupyingIncBits {
availableBits := mathutil.Min(autoid.MaxAutoRandomBits, availableBits)
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, availableBits, newAutoRandBits, oldName.O)
job.State = model.JobStateCancelled
usedBits := uint64(64 - bits.LeadingZeros64(uint64(currentIncBitsVal)))
if usedBits > newLayout.IncrementalBits {
overflowCnt := usedBits - newLayout.IncrementalBits
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg, newAutoRandBits-overflowCnt, newAutoRandBits, oldCol.Name.O)
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
return nil
}

// applyNewAutoRandomBits set auto_random bits to TableInfo and
// migrate auto_increment ID to auto_random ID if possible.
func applyNewAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo,
tblInfo *model.TableInfo, oldCol *model.ColumnInfo, newAutoRandBits uint64) error {
tblInfo.AutoRandomBits = newAutoRandBits
needMigrateFromAutoIncToAutoRand := mysql.HasAutoIncrementFlag(oldCol.Flag)
if !needMigrateFromAutoIncToAutoRand {
return nil
}
autoRandAlloc := autoid.NewAllocatorsFromTblInfo(d.store, dbInfo.ID, tblInfo).Get(autoid.AutoRandomType)
if autoRandAlloc == nil {
errMsg := fmt.Sprintf(autoid.AutoRandomAllocatorNotFound, dbInfo.Name.O, tblInfo.Name.O)
return ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
nextAutoIncID, err := m.GetAutoTableID(dbInfo.ID, tblInfo.ID)
if err != nil {
return errors.Trace(err)
}
err = autoRandAlloc.Rebase(tblInfo.ID, nextAutoIncID, false)
if err != nil {
return errors.Trace(err)
}
if err := m.CleanAutoID(dbInfo.ID, tblInfo.ID); err != nil {
return errors.Trace(err)
}
return nil
}

Expand Down
14 changes: 14 additions & 0 deletions ddl/db_integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2348,3 +2348,17 @@ func (s *testIntegrationSuite7) TestDuplicateErrorMessage(c *C) {
}
}
}

func (s *testIntegrationSuite7) TestAutoIncrementAllocator(c *C) {
tk := testkit.NewTestKit(c, s.store)
defer config.RestoreFunc()()
config.UpdateGlobal(func(conf *config.Config) {
conf.AlterPrimaryKey = false
})
tk.MustExec("drop database if exists test_create_table_option_auto_inc;")
tk.MustExec("create database test_create_table_option_auto_inc;")
tk.MustExec("use test_create_table_option_auto_inc;")

tk.MustExec("create table t (a bigint primary key) auto_increment = 10;")
tk.MustExec("alter table t auto_increment = 10;")
}
79 changes: 42 additions & 37 deletions ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -1574,14 +1574,16 @@ func (d *ddl) CreateTableWithInfo(
if tbInfo.AutoIncID > 1 {
// Default tableAutoIncID base is 0.
// If the first ID is expected to greater than 1, we need to do rebase.
if err = d.handleAutoIncID(tbInfo, schema.ID, autoid.RowIDAllocType); err != nil {
newEnd := tbInfo.AutoIncID - 1
if err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.RowIDAllocType); err != nil {
return errors.Trace(err)
}
}
if tbInfo.AutoRandID > 1 {
// Default tableAutoRandID base is 0.
// If the first ID is expected to greater than 1, we need to do rebase.
err = d.handleAutoIncID(tbInfo, schema.ID, autoid.AutoRandomType)
newEnd := tbInfo.AutoRandID - 1
err = d.handleAutoIncID(tbInfo, schema.ID, newEnd, autoid.AutoRandomType)
}
}

Expand Down Expand Up @@ -1884,22 +1886,11 @@ func checkCharsetAndCollation(cs string, co string) error {

// handleAutoIncID handles auto_increment option in DDL. It creates a ID counter for the table and initiates the counter to a proper value.
// For example if the option sets auto_increment to 10. The counter will be set to 9. So the next allocated ID will be 10.
func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, tp autoid.AllocatorType) error {
func (d *ddl) handleAutoIncID(tbInfo *model.TableInfo, schemaID int64, newEnd int64, tp autoid.AllocatorType) error {
allocs := autoid.NewAllocatorsFromTblInfo(d.store, schemaID, tbInfo)
tbInfo.State = model.StatePublic
tb, err := table.TableFromMeta(allocs, tbInfo)
if err != nil {
return errors.Trace(err)
}
// The operation of the minus 1 to make sure that the current value doesn't be used,
// the next Alloc operation will get this value.
// Its behavior is consistent with MySQL.
if tp == autoid.RowIDAllocType {
if err = tb.RebaseAutoID(nil, tbInfo.AutoIncID-1, false, tp); err != nil {
return errors.Trace(err)
}
} else {
if err = tb.RebaseAutoID(nil, tbInfo.AutoRandID-1, false, tp); err != nil {
if alloc := allocs.Get(tp); alloc != nil {
err := alloc.Rebase(tbInfo.ID, newEnd, false)
if err != nil {
return errors.Trace(err)
}
}
Expand Down Expand Up @@ -2240,16 +2231,18 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6
actionType = model.ActionRebaseAutoID
}

autoID, err := t.Allocators(ctx).Get(tp).NextGlobalAutoID(t.Meta().ID)
if err != nil {
return errors.Trace(err)
if alloc := t.Allocators(ctx).Get(tp); alloc != nil {
autoID, err := alloc.NextGlobalAutoID(t.Meta().ID)
if err != nil {
return errors.Trace(err)
}
// If newBase < autoID, we need to do a rebase before returning.
// Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000.
// If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B,
// and TiDB-B finds 100 < 30001 but returns without any handling,
// then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user.
newBase = int64(mathutil.MaxUint64(uint64(newBase), uint64(autoID)))
}
// If newBase < autoID, we need to do a rebase before returning.
// Assume there are 2 TiDB servers: TiDB-A with allocator range of 0 ~ 30000; TiDB-B with allocator range of 30001 ~ 60000.
// If the user sends SQL `alter table t1 auto_increment = 100` to TiDB-B,
// and TiDB-B finds 100 < 30001 but returns without any handling,
// then TiDB-A may still allocate 99 for auto_increment column. This doesn't make sense for the user.
newBase = int64(mathutil.MaxUint64(uint64(newBase), uint64(autoID)))
job := &model.Job{
SchemaID: schema.ID,
TableID: t.Meta().ID,
Expand Down Expand Up @@ -3119,9 +3112,8 @@ func checkColumnWithIndexConstraint(tbInfo *model.TableInfo, originalCol, newCol
}

func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNewColumn *ast.ColumnDef) (uint64, error) {
// Disallow add/drop actions on auto_random.
var oldRandBits uint64
if tableInfo.PKIsHandle && (tableInfo.GetPkName().L == originCol.Name.L) {
if originCol.IsPKHandleColumn(tableInfo) {
oldRandBits = tableInfo.AutoRandomBits
}
newRandBits, err := extractAutoRandomBitsFromColDef(specNewColumn)
Expand All @@ -3131,25 +3123,38 @@ func checkAutoRandom(tableInfo *model.TableInfo, originCol *table.Column, specNe
switch {
case oldRandBits == newRandBits:
break
case oldRandBits == 0 || newRandBits == 0:
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg)
case autoid.MaxAutoRandomBits < newRandBits:
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg,
autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
case oldRandBits < newRandBits:
addingAutoRandom := oldRandBits == 0
if addingAutoRandom {
convFromAutoInc := mysql.HasAutoIncrementFlag(originCol.Flag) && originCol.IsPKHandleColumn(tableInfo)
if !convFromAutoInc {
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterChangeFromAutoInc)
}
}
if autoid.MaxAutoRandomBits < newRandBits {
errMsg := fmt.Sprintf(autoid.AutoRandomOverflowErrMsg,
autoid.MaxAutoRandomBits, newRandBits, specNewColumn.Name.Name.O)
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(errMsg)
}
break // Increasing auto_random shard bits is allowed.
case oldRandBits > newRandBits:
if newRandBits == 0 {
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomAlterErrMsg)
}
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomDecreaseBitErrMsg)
}

if oldRandBits != 0 {
modifyingAutoRandCol := oldRandBits > 0 || newRandBits > 0
if modifyingAutoRandCol {
// Disallow changing the column field type.
if originCol.Tp != specNewColumn.Tp.Tp {
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomModifyColTypeErrMsg)
}
// Disallow changing auto_increment on auto_random column.
if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) != mysql.HasAutoIncrementFlag(originCol.Flag) {
if originCol.Tp != mysql.TypeLonglong {
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(fmt.Sprintf(autoid.AutoRandomOnNonBigIntColumn, types.TypeStr(originCol.Tp)))
}
// Disallow changing from auto_random to auto_increment column.
if containsColumnOption(specNewColumn, ast.ColumnOptionAutoIncrement) {
return 0, ErrInvalidAutoRandom.GenWithStackByArgs(autoid.AutoRandomIncompatibleWithAutoIncErrMsg)
}
// Disallow specifying a default value on auto_random column.
Expand Down
2 changes: 1 addition & 1 deletion ddl/ddl_worker.go
Original file line number Diff line number Diff line change
Expand Up @@ -689,7 +689,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64,
case model.ActionDropColumn:
ver, err = onDropColumn(t, job)
case model.ActionModifyColumn:
ver, err = w.onModifyColumn(t, job)
ver, err = w.onModifyColumn(d, t, job)
case model.ActionSetDefaultValue:
ver, err = onSetDefaultValue(t, job)
case model.ActionAddIndex:
Expand Down
65 changes: 62 additions & 3 deletions ddl/serial_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -835,6 +835,9 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
assertAlterValue := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomAlterErrMsg)
}
assertOnlyChangeFromAutoIncPK := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomAlterChangeFromAutoInc)
}
assertDecreaseBitErr := func(sql string) {
assertInvalidAutoRandomErr(sql, autoid.AutoRandomDecreaseBitErrMsg)
}
Expand Down Expand Up @@ -952,11 +955,11 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
assertAlterValue("alter table t change column c d bigint")
})
mustExecAndDrop("create table t (a bigint primary key)", func() {
assertAlterValue("alter table t modify column a bigint auto_random(3)")
assertOnlyChangeFromAutoIncPK("alter table t modify column a bigint auto_random(3)")
})
mustExecAndDrop("create table t (a bigint, b bigint, primary key(a, b))", func() {
assertAlterValue("alter table t modify column a bigint auto_random(3)")
assertAlterValue("alter table t modify column b bigint auto_random(3)")
assertOnlyChangeFromAutoIncPK("alter table t modify column a bigint auto_random(3)")
assertOnlyChangeFromAutoIncPK("alter table t modify column b bigint auto_random(3)")
})

// Decrease auto_random bits is not allowed.
Expand Down Expand Up @@ -1025,6 +1028,62 @@ func (s *testSerialSuite) TestAutoRandom(c *C) {
})
}

func (s *testSerialSuite) TestAutoRandomChangeFromAutoInc(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
tk.MustExec("set @@tidb_allow_remove_auto_inc = 1;")

// Basic usages.
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint auto_increment primary key);")
tk.MustExec("insert into t values (), (), ();")
tk.MustExec("alter table t modify column a bigint auto_random(3);")
tk.MustExec("insert into t values (), (), ();")
rows := tk.MustQuery("show table t next_row_id;").Rows()
c.Assert(len(rows), Equals, 1, Commentf("query result: %v", rows))
c.Assert(len(rows[0]), Equals, 5, Commentf("query result: %v", rows))
c.Assert(rows[0][4], Equals, "AUTO_RANDOM")

// Changing from auto_inc unique key is not allowed.
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint auto_increment unique key);")
tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom)
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint auto_increment unique key, b bigint auto_random primary key);")
tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom)

// Changing from non-auto-inc column is not allowed.
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint);")
tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom)
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint primary key);")
tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom)

// Changing from non BIGINT auto_inc pk column is not allowed.
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a int auto_increment primary key);")
tk.MustGetErrCode("alter table t modify column a int auto_random;", errno.ErrInvalidAutoRandom)
tk.MustGetErrCode("alter table t modify column a bigint auto_random;", errno.ErrInvalidAutoRandom)

// Changing from auto_random to auto_increment is not allowed.
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint auto_random primary key);")
// "Unsupported modify column: can't set auto_increment"
tk.MustGetErrCode("alter table t modify column a bigint auto_increment;", errno.ErrUnsupportedDDLOperation)

// Large auto_increment number overflows auto_random.
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint auto_increment primary key);")
tk.MustExec("insert into t values (1<<(64-5));")
// "max allowed auto_random shard bits is 3, but got 4 on column `a`"
tk.MustGetErrCode("alter table t modify column a bigint auto_random(4);", errno.ErrInvalidAutoRandom)
tk.MustExec("drop table if exists t;")
tk.MustExec("create table t (a bigint auto_increment primary key);")
tk.MustExec("insert into t values (1<<(64-6));")
tk.MustExec("alter table t modify column a bigint auto_random(4);")
}

func (s *testSerialSuite) TestAutoRandomIncBitsIncrementAndOffset(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("create database if not exists auto_random_db")
Expand Down
Loading

0 comments on commit 9b2443a

Please sign in to comment.