diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index e4cfe86f4d89c..289254f3b2c34 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -16,6 +16,7 @@ package ddl_test import ( "context" "fmt" + "math" "strconv" "strings" "sync/atomic" @@ -2623,6 +2624,95 @@ func (s *testSerialDBSuite1) TestAutoIncrementTableOption(c *C) { tk.MustQuery("select * from t;").Check(testkit.Rows("12345678901234567890")) } +func (s *testIntegrationSuite3) TestAutoIncrementForce(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("drop database if exists auto_inc_force;") + tk.MustExec("create database auto_inc_force;") + tk.MustExec("use auto_inc_force;") + getNextGlobalID := func() uint64 { + gidStr := tk.MustQuery("show table t next_row_id").Rows()[0][3] + gid, err := strconv.ParseUint(gidStr.(string), 10, 64) + c.Assert(err, IsNil) + return gid + } + // Rebase _tidb_row_id. + tk.MustExec("create table t (a int);") + tk.MustExec("insert into t values (1),(2);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 1", "2 2")) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_increment = 1;") + c.Assert(getNextGlobalID(), Equals, uint64(1)) + // inserting new rows can overwrite the existing data. + tk.MustExec("insert into t values (3);") + tk.MustExec("insert into t values (3);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "3 2")) + + // Rebase auto_increment. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a int primary key auto_increment, b int);") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (100000000, 1);") + tk.MustExec("delete from t where a = 100000000;") + c.Assert(getNextGlobalID(), Greater, uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t /*T![force_inc] force */ auto_increment = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t /*T![force_inc] force */ auto_increment = 2;") + c.Assert(getNextGlobalID(), Equals, uint64(2)) + tk.MustExec("insert into t(b) values (2);") + tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) + + // Rebase auto_random. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint primary key auto_random(5));") + tk.MustExec("insert into t values ();") + tk.MustExec("set @@allow_auto_random_explicit_insert = true") + tk.MustExec("insert into t values (100000000);") + tk.MustExec("delete from t where a = 100000000;") + c.Assert(getNextGlobalID(), Greater, uint64(100000000)) + // Cannot set next global ID to 0. + tk.MustGetErrCode("alter table t force auto_random_base = 0;", errno.ErrAutoincReadFailed) + tk.MustExec("alter table t force auto_random_base = 2;") + c.Assert(getNextGlobalID(), Equals, uint64(2)) + tk.MustExec("insert into t values ();") + tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) + + // Change next global ID. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint primary key auto_increment);") + tk.MustExec("insert into t values (1);") + bases := []uint64{1, 65535, 10, math.MaxUint64, math.MaxInt64 + 1, 1, math.MaxUint64, math.MaxInt64, 2} + lastBase := fmt.Sprintf("%d", bases[len(bases)-1]) + for _, b := range bases { + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + c.Assert(getNextGlobalID(), Equals, b) + } + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows("1", lastBase)) + // Force alter unsigned int auto_increment column. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (a bigint unsigned primary key auto_increment);") + for _, b := range bases { + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + c.Assert(getNextGlobalID(), Equals, b) + tk.MustExec("insert into t values ();") + tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) + tk.MustExec("delete from t;") + } + + // Force alter with @@auto_increment_increment and @@auto_increment_offset. + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t(a int key auto_increment);") + tk.MustExec("set @@auto_increment_offset=2;") + tk.MustExec("set @@auto_increment_increment = 11;") + tk.MustExec("insert into t values (500);") + tk.MustExec("alter table t force auto_increment=100;") + tk.MustExec("insert into t values (), ();") + tk.MustQuery("select * from t;").Check(testkit.Rows("101", "112", "500")) + tk.MustQuery("select * from t order by a;").Check(testkit.Rows("101", "112", "500")) + tk.MustExec("drop table if exists t;") +} + func (s *testIntegrationSuite3) TestIssue20490(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test;") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index af6cc2fd13ae7..91fb1a215e337 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2532,7 +2532,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } err = d.ShardRowID(ctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.RowIDAllocType) + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.RowIDAllocType, opt.BoolValue) case ast.TableOptionAutoIdCache: if opt.UintValue > uint64(math.MaxInt64) { // TODO: Refine this error. @@ -2540,7 +2540,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A } err = d.AlterTableAutoIDCache(ctx, ident, int64(opt.UintValue)) case ast.TableOptionAutoRandomBase: - err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.AutoRandomType) + err = d.RebaseAutoID(ctx, ident, int64(opt.UintValue), autoid.AutoRandomType, opt.BoolValue) case ast.TableOptionComment: spec.Comment = opt.StrValue err = d.AlterTableComment(ctx, ident, spec) @@ -2596,7 +2596,7 @@ func (d *ddl) AlterTable(ctx sessionctx.Context, ident ast.Ident, specs []*ast.A return nil } -func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp autoid.AllocatorType) error { +func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int64, tp autoid.AllocatorType, force bool) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, ident) if err != nil { return errors.Trace(err) @@ -2625,17 +2625,11 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 actionType = model.ActionRebaseAutoID } - if alloc := t.Allocators(ctx).Get(tp); alloc != nil { - autoID, err := alloc.NextGlobalAutoID(t.Meta().ID) + if !force { + newBase, err = adjustNewBaseToNextGlobalID(ctx, t, tp, newBase) if err != nil { - return errors.Trace(err) + return 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))) } job := &model.Job{ SchemaID: schema.ID, @@ -2643,13 +2637,30 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 SchemaName: schema.Name.L, Type: actionType, BinlogInfo: &model.HistoryInfo{}, - Args: []interface{}{newBase}, + Args: []interface{}{newBase, force}, } err = d.doDDLJob(ctx, job) err = d.callHookOnChanged(err) return errors.Trace(err) } +func adjustNewBaseToNextGlobalID(ctx sessionctx.Context, t table.Table, tp autoid.AllocatorType, newBase int64) (int64, error) { + alloc := t.Allocators(ctx).Get(tp) + if alloc == nil { + return newBase, nil + } + autoID, err := alloc.NextGlobalAutoID(t.Meta().ID) + if err != nil { + return newBase, 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. + return int64(mathutil.MaxUint64(uint64(newBase), uint64(autoID))), nil +} + // ShardRowID shards the implicit row ID by adding shard value to the row ID's first few bits. func (d *ddl) ShardRowID(ctx sessionctx.Context, tableIdent ast.Ident, uVal uint64) error { schema, t, err := d.getSchemaAndTableByIdent(ctx, tableIdent) diff --git a/ddl/table.go b/ddl/table.go index a3962621e5c63..336471c04bd62 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -555,8 +555,11 @@ func onRebaseAutoRandomType(store kv.Storage, t *meta.Meta, job *model.Job) (ver func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.AllocatorType) (ver int64, _ error) { schemaID := job.SchemaID - var newBase int64 - err := job.DecodeArgs(&newBase) + var ( + newBase int64 + force bool + ) + err := job.DecodeArgs(&newBase, &force) if err != nil { job.State = model.JobStateCancelled return ver, errors.Trace(err) @@ -581,8 +584,13 @@ func onRebaseAutoID(store kv.Storage, t *meta.Meta, job *model.Job, tp autoid.Al if alloc := tbl.Allocators(nil).Get(tp); alloc != nil { // The next value to allocate is `newBase`. newEnd := newBase - 1 - err = alloc.Rebase(tblInfo.ID, newEnd, false) + if force { + err = alloc.ForceRebase(tblInfo.ID, newEnd) + } else { + err = alloc.Rebase(tblInfo.ID, newEnd, false) + } if err != nil { + job.State = model.JobStateCancelled return ver, errors.Trace(err) } } diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index c4447c54f8844..cf61beaff9781 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -135,6 +135,9 @@ type Allocator interface { // If allocIDs is false, it will not allocate IDs. Rebase(tableID, newBase int64, allocIDs bool) error + // ForceRebase set the next global auto ID to newBase. + ForceRebase(tableID, newBase int64) error + // RebaseSeq rebases the sequence value in number axis with tableID and the new base value. RebaseSeq(table, newBase int64) (int64, bool, error) @@ -370,16 +373,49 @@ func (alloc *allocator) Rebase(tableID, requiredBase int64, allocIDs bool) error if tableID == 0 { return errInvalidTableID.GenWithStack("Invalid tableID") } - alloc.mu.Lock() defer alloc.mu.Unlock() - if alloc.isUnsigned { return alloc.rebase4Unsigned(tableID, uint64(requiredBase), allocIDs) } return alloc.rebase4Signed(tableID, requiredBase, allocIDs) } +// ForceRebase implements autoid.Allocator ForceRebase interface. +func (alloc *allocator) ForceRebase(tableID, requiredBase int64) error { + if tableID <= 0 { + return errInvalidTableID.GenWithStack("Invalid tableID") + } + if requiredBase == -1 { + return ErrAutoincReadFailed.GenWithStack("Cannot force rebase the next global ID to '0'") + } + alloc.mu.Lock() + defer alloc.mu.Unlock() + startTime := time.Now() + err := kv.RunInNewTxn(context.Background(), alloc.store, true, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + currentEnd, err1 := GetAutoID(m, alloc.dbID, tableID, alloc.allocType) + if err1 != nil { + return err1 + } + var step int64 + if !alloc.isUnsigned { + step = requiredBase - currentEnd + } else { + uRequiredBase, uCurrentEnd := uint64(requiredBase), uint64(currentEnd) + step = int64(uRequiredBase - uCurrentEnd) + } + _, err1 = GenerateAutoID(m, alloc.dbID, tableID, step, alloc.allocType) + return err1 + }) + metrics.AutoIDHistogram.WithLabelValues(metrics.TableAutoIDRebase, metrics.RetLabel(err)).Observe(time.Since(startTime).Seconds()) + if err != nil { + return err + } + alloc.base, alloc.end = requiredBase, requiredBase + return nil +} + // Rebase implements autoid.Allocator RebaseSeq interface. // The return value is quite same as expression function, bool means whether it should be NULL, // here it will be used in setval expression function (true meaning the set value has been satisfied, return NULL). diff --git a/meta/autoid/memid.go b/meta/autoid/memid.go index d9a6be9f6058a..39db68e3bc33d 100644 --- a/meta/autoid/memid.go +++ b/meta/autoid/memid.go @@ -98,6 +98,12 @@ func (alloc *inMemoryAllocator) Rebase(tableID, requiredBase int64, allocIDs boo return nil } +// ForceRebase implements autoid.Allocator ForceRebase interface. +func (alloc *inMemoryAllocator) ForceRebase(tableID, requiredBase int64) error { + alloc.base = requiredBase + return nil +} + func (alloc *inMemoryAllocator) alloc4Signed(n uint64, increment, offset int64) (int64, int64, error) { // Check offset rebase if necessary. if offset-1 > alloc.base { diff --git a/sessionctx/binloginfo/binloginfo_test.go b/sessionctx/binloginfo/binloginfo_test.go index 6a586615bd8a0..4e52d2b100da6 100644 --- a/sessionctx/binloginfo/binloginfo_test.go +++ b/sessionctx/binloginfo/binloginfo_test.go @@ -666,6 +666,14 @@ func (s *testBinlogSuite) TestAddSpecialComment(c *C) { "create table t1 (id int, a varchar(255) key clustered);", "create table t1 (id int, a varchar(255) key /*T![clustered_index] clustered */ );", }, + { + "alter table t force auto_increment = 12;", + "alter table t /*T![force_inc] force */ auto_increment = 12;", + }, + { + "alter table t force, auto_increment = 12;", + "alter table t force, auto_increment = 12;", + }, } for _, ca := range testCase { re := binloginfo.AddSpecialComment(ca.input) diff --git a/types/parser_driver/special_cmt_ctrl.go b/types/parser_driver/special_cmt_ctrl.go index 6fa7ebf67d6cd..31505ebc40f71 100644 --- a/types/parser_driver/special_cmt_ctrl.go +++ b/types/parser_driver/special_cmt_ctrl.go @@ -35,6 +35,7 @@ func init() { parser.SpecialCommentsController.Register(string(FeatureIDAutoIDCache)) parser.SpecialCommentsController.Register(string(FeatureIDAutoRandomBase)) parser.SpecialCommentsController.Register(string(FeatureClusteredIndex)) + parser.SpecialCommentsController.Register(string(FeatureForceAutoInc)) } // SpecialCommentVersionPrefix is the prefix of TiDB executable comments. @@ -61,6 +62,8 @@ const ( FeatureIDAutoRandomBase featureID = "auto_rand_base" // FeatureClusteredIndex is the `clustered_index` feature. FeatureClusteredIndex featureID = "clustered_index" + // FeatureForceAutoInc is the `force auto_increment` feature. + FeatureForceAutoInc featureID = "force_inc" ) // FeatureIDPatterns is used to record special comments patterns. @@ -69,4 +72,5 @@ var FeatureIDPatterns = map[featureID]*regexp.Regexp{ FeatureIDAutoIDCache: regexp.MustCompile(`(?P(?i)AUTO_ID_CACHE\s*=?\s*\d+\s*)`), FeatureIDAutoRandomBase: regexp.MustCompile(`(?P(?i)AUTO_RANDOM_BASE\s*=?\s*\d+\s*)`), FeatureClusteredIndex: regexp.MustCompile(`(?i)(PRIMARY)?\s+KEY(\s*\(.*\))?\s+(?P(NON)?CLUSTERED\b)`), + FeatureForceAutoInc: regexp.MustCompile(`(?P(?i)FORCE)\b\s*AUTO_INCREMENT\s*`), }