diff --git a/autoid_service/BUILD.bazel b/autoid_service/BUILD.bazel index df3d361d412ed..c973afd85c4e7 100644 --- a/autoid_service/BUILD.bazel +++ b/autoid_service/BUILD.bazel @@ -11,6 +11,7 @@ go_library( "//meta", "//metrics", "//owner", + "//parser/model", "//util/logutil", "//util/mathutil", "@com_github_pingcap_errors//:errors", diff --git a/autoid_service/autoid.go b/autoid_service/autoid.go index f2836fc80fe85..fd0feae76f6bf 100644 --- a/autoid_service/autoid.go +++ b/autoid_service/autoid.go @@ -28,6 +28,7 @@ import ( "github.com/pingcap/tidb/meta" "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/owner" + "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/util/logutil" "github.com/pingcap/tidb/util/mathutil" clientv3 "go.etcd.io/etcd/client/v3" @@ -76,7 +77,7 @@ func (alloc *autoIDValue) alloc4Unsigned(ctx context.Context, store kv.Storage, ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) var err1 error newBase, err1 = idAcc.Get() if err1 != nil { @@ -137,7 +138,7 @@ func (alloc *autoIDValue) alloc4Signed(ctx context.Context, ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) var err1 error newBase, err1 = idAcc.Get() if err1 != nil { @@ -188,7 +189,7 @@ func (alloc *autoIDValue) rebase4Unsigned(ctx context.Context, startTime := time.Now() ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) currentEnd, err1 := idAcc.Get() if err1 != nil { return err1 @@ -221,7 +222,7 @@ func (alloc *autoIDValue) rebase4Signed(ctx context.Context, store kv.Storage, d var newBase, newEnd int64 ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) currentEnd, err1 := idAcc.Get() if err1 != nil { return err1 @@ -451,7 +452,7 @@ func (s *Service) allocAutoID(ctx context.Context, req *autoid.AutoIDRequest) (* func (alloc *autoIDValue) forceRebase(ctx context.Context, store kv.Storage, dbID, tblID, requiredBase int64, isUnsigned bool) error { ctx = kv.WithInternalSourceType(ctx, kv.InternalTxnMeta) err := kv.RunInNewTxn(ctx, store, true, func(ctx context.Context, txn kv.Transaction) error { - idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).RowID() + idAcc := meta.NewMeta(txn).GetAutoIDAccessors(dbID, tblID).IncrementID(model.TableInfoVersion5) currentEnd, err1 := idAcc.Get() if err1 != nil { return err1 diff --git a/br/pkg/lightning/backend/kv/allocator.go b/br/pkg/lightning/backend/kv/allocator.go index 02f46ea8c7e36..14703e1143a45 100644 --- a/br/pkg/lightning/backend/kv/allocator.go +++ b/br/pkg/lightning/backend/kv/allocator.go @@ -34,6 +34,7 @@ type panickingAllocator struct { func NewPanickingAllocators(base int64) autoid.Allocators { sharedBase := &base return autoid.NewAllocators( + false, &panickingAllocator{base: sharedBase, ty: autoid.RowIDAllocType}, &panickingAllocator{base: sharedBase, ty: autoid.AutoIncrementType}, &panickingAllocator{base: sharedBase, ty: autoid.AutoRandomType}, diff --git a/ddl/column.go b/ddl/column.go index 96ab745377a92..9f5174737915a 100644 --- a/ddl/column.go +++ b/ddl/column.go @@ -1543,7 +1543,7 @@ func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, return nil } idAcc := m.GetAutoIDAccessors(dbInfo.ID, tblInfo.ID) - err := checkNewAutoRandomBits(idAcc, oldCol, newCol, newAutoRandBits, tblInfo.AutoRandomRangeBits, tblInfo.Version) + err := checkNewAutoRandomBits(idAcc, oldCol, newCol, newAutoRandBits, tblInfo.AutoRandomRangeBits, tblInfo.SepAutoInc()) if err != nil { return err } @@ -1552,13 +1552,17 @@ func checkAndApplyAutoRandomBits(d *ddlCtx, m *meta.Meta, dbInfo *model.DBInfo, // checkNewAutoRandomBits checks whether the new auto_random bits number can cause overflow. func checkNewAutoRandomBits(idAccessors meta.AutoIDAccessors, oldCol *model.ColumnInfo, - newCol *model.ColumnInfo, newShardBits, newRangeBits uint64, tblVer uint16) error { + newCol *model.ColumnInfo, newShardBits, newRangeBits uint64, sepAutoInc bool) error { shardFmt := autoid.NewShardIDFormat(&newCol.FieldType, newShardBits, newRangeBits) idAcc := idAccessors.RandomID() convertedFromAutoInc := mysql.HasAutoIncrementFlag(oldCol.GetFlag()) if convertedFromAutoInc { - idAcc = idAccessors.IncrementID(tblVer) + if sepAutoInc { + idAcc = idAccessors.IncrementID(model.TableInfoVersion5) + } else { + idAcc = idAccessors.RowID() + } } // Generate a new auto ID first to prevent concurrent update in DML. _, err := idAcc.Inc(1) diff --git a/ddl/db_integration_test.go b/ddl/db_integration_test.go index b25aa87f2fce8..6d2e5cf39c468 100644 --- a/ddl/db_integration_test.go +++ b/ddl/db_integration_test.go @@ -2926,94 +2926,84 @@ func TestAutoIncrementForce(t *testing.T) { return gid } - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Rebase _tidb_row_id. - tk.MustExec("create table t (a int)" + str) - tk.MustExec("alter table t force auto_increment = 2;") - tk.MustExec("insert into t values (1),(2);") - tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) - // 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;") - require.Equal(t, uint64(1), getNextGlobalID()) - // inserting new rows can overwrite the existing data. - tk.MustExec("insert into t values (3);") - require.Equal(t, "[kv:1062]Duplicate entry '2' for key 't.PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) - tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) - tk.MustExec("drop table if exists t;") - } + // Rebase _tidb_row_id. + tk.MustExec("create table t (a int)") + tk.MustExec("alter table t force auto_increment = 2;") + tk.MustExec("insert into t values (1),(2);") + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("1 2", "2 3")) + // 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;") + require.Equal(t, uint64(1), getNextGlobalID()) + // inserting new rows can overwrite the existing data. + tk.MustExec("insert into t values (3);") + require.Equal(t, "[kv:1062]Duplicate entry '2' for key 't.PRIMARY'", tk.ExecToErr("insert into t values (3);").Error()) + tk.MustQuery("select a, _tidb_rowid from t;").Check(testkit.Rows("3 1", "1 2", "2 3")) + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Rebase auto_increment. - tk.MustExec("create table t (a int primary key auto_increment, b int)" + str) - tk.MustExec("insert into t values (1, 1);") - tk.MustExec("insert into t values (100000000, 1);") - tk.MustExec("delete from t where a = 100000000;") - require.Greater(t, getNextGlobalID(), 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;") - require.Equal(t, uint64(2), getNextGlobalID()) - tk.MustExec("insert into t(b) values (2);") - tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) - tk.MustExec("drop table if exists t;") - } + // Rebase auto_increment. + 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;") + require.Greater(t, getNextGlobalID(), 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;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t(b) values (2);") + tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Rebase auto_random. - tk.MustExec("create table t (a bigint primary key auto_random(5))" + str) - 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;") - require.Greater(t, getNextGlobalID(), 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;") - require.Equal(t, uint64(2), getNextGlobalID()) - tk.MustExec("insert into t values ();") - tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) - tk.MustExec("drop table if exists t;") - } + // Rebase auto_random. + 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;") + require.Greater(t, getNextGlobalID(), 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;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t values ();") + tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Change next global ID. - tk.MustExec("create table t (a bigint primary key auto_increment)" + str) - 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 { - fmt.Println("execute alter table force increment to ==", b) - tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - require.Equal(t, b, getNextGlobalID()) - } + // Change next global ID. + 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 { + fmt.Println("execute alter table force increment to ==", b) + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + require.Equal(t, b, getNextGlobalID()) + } + 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)) + require.Equal(t, b, getNextGlobalID()) 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)" + str) - for _, b := range bases { - tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) - require.Equal(t, b, getNextGlobalID()) - tk.MustExec("insert into t values ();") - tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) - tk.MustExec("delete from t;") - } - tk.MustExec("drop table if exists t;") + tk.MustQuery("select a from t;").Check(testkit.Rows(fmt.Sprintf("%d", b))) + tk.MustExec("delete from t;") } + tk.MustExec("drop table if exists t;") - for _, str := range []string{"", " AUTO_ID_CACHE 1"} { - // Force alter with @@auto_increment_increment and @@auto_increment_offset. - tk.MustExec("create table t(a int key auto_increment)" + str) - 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;") - } + // Force alter with @@auto_increment_increment and @@auto_increment_offset. + 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;") // Check for warning in case we can't set the auto_increment to the desired value tk.MustExec("create table t(a int primary key auto_increment)") @@ -3033,6 +3023,147 @@ func TestAutoIncrementForce(t *testing.T) { tk.MustExec("drop table t") } +func TestAutoIncrementForceAutoIDCache(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, 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) + require.NoError(t, err) + return gid + } + + // Rebase _tidb_row_id. + tk.MustExec("create table t (a int) AUTO_ID_CACHE 1") + tk.MustExec("alter table t force auto_increment = 2;") + 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.MustExec("alter table t force auto_increment = 0;") + tk.MustExec("alter table t force auto_increment = 1;") + require.Equal(t, uint64(3), getNextGlobalID()) + // 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("1 1", "2 2", "3 3", "3 4")) + tk.MustExec("drop table if exists t;") + + // When AUTO_ID_CACHE is 1, row id and auto increment id use separate allocator, so the behaviour differs. + // "Alter table t force auto_increment" has no effect on row id. + tk.MustExec("create table t (a int) AUTO_ID_CACHE 1") + tk.MustExec("alter table t force auto_increment = 2;") + 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.MustExec("alter table t force auto_increment = 0;") + tk.MustExec("alter table t force auto_increment = 1;") + require.Equal(t, uint64(3), getNextGlobalID()) + // 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("1 1", "2 2", "3 3", "3 4")) + tk.MustExec("drop table if exists t;") + + // Rebase auto_increment. + tk.MustExec("create table t (a int primary key auto_increment, b int) AUTO_ID_CACHE 1") + tk.MustExec("insert into t values (1, 1);") + tk.MustExec("insert into t values (100000000, 1);") + tk.MustExec("delete from t where a = 100000000;") + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + "auto_inc_force t a 100000001 AUTO_INCREMENT", + )) + // 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;") + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + "auto_inc_force t a 2 AUTO_INCREMENT", + )) + + tk.MustExec("insert into t(b) values (2);") + tk.MustQuery("select a, b from t;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table if exists t;") + + // Rebase auto_random. + tk.MustExec("create table t (a bigint primary key auto_random(5)) AUTO_ID_CACHE 1") + 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;") + require.Greater(t, getNextGlobalID(), 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;") + require.Equal(t, uint64(2), getNextGlobalID()) + tk.MustExec("insert into t values ();") + tk.MustQuery("select (a & 3) from t order by 1;").Check(testkit.Rows("1", "2")) + tk.MustExec("drop table if exists t;") + + // Change next global ID. + tk.MustExec("create table t (a bigint primary key auto_increment) AUTO_ID_CACHE 1") + 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 { + fmt.Println("execute alter table force increment to ==", b) + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + fmt.Sprintf("auto_inc_force t a %d AUTO_INCREMENT", 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) AUTO_ID_CACHE 1") + for _, b := range bases { + tk.MustExec(fmt.Sprintf("alter table t force auto_increment = %d;", b)) + tk.MustQuery("show table t next_row_id").Check(testkit.Rows( + "auto_inc_force t a 1 _TIDB_ROWID", + fmt.Sprintf("auto_inc_force t a %d AUTO_INCREMENT", 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;") + } + tk.MustExec("drop table if exists t;") + + // Force alter with @@auto_increment_increment and @@auto_increment_offset. + tk.MustExec("create table t(a int key auto_increment) AUTO_ID_CACHE 1") + 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;") + + // Check for warning in case we can't set the auto_increment to the desired value + tk.MustExec("create table t(a int primary key auto_increment) AUTO_ID_CACHE 1") + tk.MustExec("insert into t values (200)") + tk.MustQuery("show create table t").Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL AUTO_INCREMENT,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=1 */")) + tk.MustExec("alter table t auto_increment=100;") + tk.MustQuery("show warnings").Check(testkit.Rows("Warning 1105 Can't reset AUTO_INCREMENT to 100 without FORCE option, using 201 instead")) + tk.MustExec("insert into t values ()") + tk.MustQuery("select * from t").Check(testkit.Rows("200", "211")) + tk.MustQuery("show create table t").Check(testkit.Rows( + "t CREATE TABLE `t` (\n" + + " `a` int(11) NOT NULL AUTO_INCREMENT,\n" + + " PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */\n" + + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=1 */")) + tk.MustExec("drop table t") +} + func TestIssue20490(t *testing.T) { store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) diff --git a/ddl/db_test.go b/ddl/db_test.go index ce2ea307e97af..5990df4634a68 100644 --- a/ddl/db_test.go +++ b/ddl/db_test.go @@ -898,11 +898,11 @@ func TestAutoIncrementIDOnTemporaryTable(t *testing.T) { tk.MustExec("drop table if exists global_temp_auto_id") tk.MustExec("create global temporary table global_temp_auto_id(id int primary key auto_increment) on commit delete rows") tk.MustExec("begin") - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 1 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 1 _TIDB_ROWID")) tk.MustExec("insert into global_temp_auto_id value(null)") tk.MustQuery("select @@last_insert_id").Check(testkit.Rows("1")) tk.MustQuery("select id from global_temp_auto_id").Check(testkit.Rows("1")) - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 2 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 2 _TIDB_ROWID")) tk.MustExec("commit") tk.MustExec("drop table global_temp_auto_id") @@ -914,12 +914,12 @@ func TestAutoIncrementIDOnTemporaryTable(t *testing.T) { " `id` int(11) NOT NULL AUTO_INCREMENT,\n" + " PRIMARY KEY (`id`) /*T![clustered_index] CLUSTERED */\n" + ") ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=100 ON COMMIT DELETE ROWS")) - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 100 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 100 _TIDB_ROWID")) tk.MustExec("begin") tk.MustExec("insert into global_temp_auto_id value(null)") tk.MustQuery("select @@last_insert_id").Check(testkit.Rows("100")) tk.MustQuery("select id from global_temp_auto_id").Check(testkit.Rows("100")) - tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 101 AUTO_INCREMENT")) + tk.MustQuery("show table global_temp_auto_id next_row_id").Check(testkit.Rows("test global_temp_auto_id id 101 _TIDB_ROWID")) tk.MustExec("commit") } tk.MustExec("drop table global_temp_auto_id") diff --git a/ddl/ddl_api.go b/ddl/ddl_api.go index 505d413fff8c2..1816e0d65891d 100644 --- a/ddl/ddl_api.go +++ b/ddl/ddl_api.go @@ -2152,7 +2152,7 @@ func checkPartitionDefinitionConstraints(ctx sessionctx.Context, tbInfo *model.T // checkTableInfoValid uses to check table info valid. This is used to validate table info. func checkTableInfoValid(tblInfo *model.TableInfo) error { - _, err := tables.TableFromMeta(nil, tblInfo) + _, err := tables.TableFromMeta(autoid.NewAllocators(false), tblInfo) if err != nil { return err } @@ -2493,7 +2493,13 @@ func (d *ddl) createTableWithInfoPost( // Default tableAutoIncID base is 0. // If the first ID is expected to greater than 1, we need to do rebase. newEnd := tbInfo.AutoIncID - 1 - if err = d.handleAutoIncID(tbInfo, schemaID, newEnd, autoid.RowIDAllocType); err != nil { + var allocType autoid.AllocatorType + if tbInfo.SepAutoInc() { + allocType = autoid.AutoIncrementType + } else { + allocType = autoid.RowIDAllocType + } + if err = d.handleAutoIncID(tbInfo, schemaID, newEnd, allocType); err != nil { return errors.Trace(err) } } @@ -3361,7 +3367,7 @@ func (d *ddl) AlterTable(ctx context.Context, sctx sessionctx.Context, stmt *ast } err = d.ShardRowID(sctx, ident, opt.UintValue) case ast.TableOptionAutoIncrement: - err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.RowIDAllocType, opt.BoolValue) + err = d.RebaseAutoID(sctx, ident, int64(opt.UintValue), autoid.AutoIncrementType, opt.BoolValue) case ast.TableOptionAutoIdCache: if opt.UintValue > uint64(math.MaxInt64) { // TODO: Refine this error. @@ -3492,6 +3498,10 @@ func (d *ddl) RebaseAutoID(ctx sessionctx.Context, ident ast.Ident, newBase int6 actionType = model.ActionRebaseAutoRandomBase case autoid.RowIDAllocType: actionType = model.ActionRebaseAutoID + case autoid.AutoIncrementType: + actionType = model.ActionRebaseAutoID + default: + panic(fmt.Sprintf("unimplemented rebase autoid type %s", tp)) } if !force { @@ -5203,6 +5213,11 @@ func (d *ddl) AlterTableAutoIDCache(ctx sessionctx.Context, ident ast.Ident, new if err != nil { return errors.Trace(err) } + tbInfo := tb.Meta() + if (newCache == 1 && tbInfo.AutoIdCache != 1) || + (newCache != 1 && tbInfo.AutoIdCache == 1) { + return fmt.Errorf("Can't Alter AUTO_ID_CACHE between 1 and non-1, the underlying implementation is different") + } job := &model.Job{ SchemaID: schema.ID, diff --git a/ddl/ddl_worker.go b/ddl/ddl_worker.go index 1a13feb81e079..850caca29e404 100644 --- a/ddl/ddl_worker.go +++ b/ddl/ddl_worker.go @@ -1232,7 +1232,7 @@ func (w *worker) runDDLJob(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, case model.ActionTruncateTable: ver, err = onTruncateTable(d, t, job) case model.ActionRebaseAutoID: - ver, err = onRebaseRowIDType(d, t, job) + ver, err = onRebaseAutoIncrementIDType(d, t, job) case model.ActionRebaseAutoRandomBase: ver, err = onRebaseAutoRandomType(d, t, job) case model.ActionRenameTable: diff --git a/ddl/multi_schema_change_test.go b/ddl/multi_schema_change_test.go index f302737d60c3b..0f8aeca87802c 100644 --- a/ddl/multi_schema_change_test.go +++ b/ddl/multi_schema_change_test.go @@ -1177,7 +1177,7 @@ func TestMultiSchemaChangeUnsupportedType(t *testing.T) { tk.MustExec("use test;") tk.MustExec("create table t (a int, b int);") - tk.MustGetErrMsg("alter table t add column c int, auto_id_cache = 1;", + tk.MustGetErrMsg("alter table t add column c int, auto_id_cache = 10;", "[ddl:8200]Unsupported multi schema change for modify auto id cache") } diff --git a/ddl/partition.go b/ddl/partition.go index a8947d091bfc5..cf4bd7aed962f 100644 --- a/ddl/partition.go +++ b/ddl/partition.go @@ -2077,7 +2077,7 @@ func (w *worker) onExchangeTablePartition(d *ddlCtx, t *meta.Meta, job *model.Jo failpoint.Return(ver, err) } sess := newSession(se) - _, err = sess.execute(context.Background(), "insert into test.pt values (40000000)", "exchange_partition_test") + _, err = sess.execute(context.Background(), "insert ignore into test.pt values (40000000)", "exchange_partition_test") if err != nil { failpoint.Return(ver, err) } diff --git a/ddl/table.go b/ddl/table.go index a6e2d93b5ac7e..9e6fab762d3c5 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -812,8 +812,8 @@ func onTruncateTable(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ erro return ver, nil } -func onRebaseRowIDType(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { - return onRebaseAutoID(d, d.store, t, job, autoid.RowIDAllocType) +func onRebaseAutoIncrementIDType(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { + return onRebaseAutoID(d, d.store, t, job, autoid.AutoIncrementType) } func onRebaseAutoRandomType(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error) { @@ -862,7 +862,7 @@ func onRebaseAutoID(d *ddlCtx, store kv.Storage, t *meta.Meta, job *model.Job, t newBase = newBaseTemp } - if tp == autoid.RowIDAllocType { + if tp == autoid.AutoIncrementType { tblInfo.AutoIncID = newBase } else { tblInfo.AutoRandID = newBase diff --git a/ddl/table_test.go b/ddl/table_test.go index 9641c24eb0f19..e0e1c45e3b0ef 100644 --- a/ddl/table_test.go +++ b/ddl/table_test.go @@ -160,7 +160,7 @@ func testGetTableWithError(store kv.Storage, schemaID, tableID int64) (table.Tab return nil, errors.New("table not found") } alloc := autoid.NewAllocator(store, schemaID, tblInfo.ID, false, autoid.RowIDAllocType) - tbl, err := table.TableFromMeta(autoid.NewAllocators(alloc), tblInfo) + tbl, err := table.TableFromMeta(autoid.NewAllocators(false, alloc), tblInfo) if err != nil { return nil, errors.Trace(err) } diff --git a/executor/autoidtest/autoid_test.go b/executor/autoidtest/autoid_test.go index 269f3966f8a01..f52f63b9c59a2 100644 --- a/executor/autoidtest/autoid_test.go +++ b/executor/autoidtest/autoid_test.go @@ -642,14 +642,23 @@ func TestAutoIDIncrementAndOffset(t *testing.T) { tk.MustExec(`insert into io(b) values (null),(null),(null)`) // AutoID allocation will take increment and offset into consideration. tk.MustQuery(`select b from io`).Check(testkit.Rows("10", "12", "14")) - // HandleID allocation will ignore the increment and offset. - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) + if str == "" { + // HandleID allocation will ignore the increment and offset. + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("15", "16", "17")) + } else { + // Separate row id and auto inc id, increment and offset works on auto inc id + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("1", "2", "3")) + } tk.MustExec(`delete from io`) tk.Session().GetSessionVars().AutoIncrementIncrement = 10 tk.MustExec(`insert into io(b) values (null),(null),(null)`) tk.MustQuery(`select b from io`).Check(testkit.Rows("20", "30", "40")) - tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) + if str == "" { + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("41", "42", "43")) + } else { + tk.MustQuery(`select _tidb_rowid from io`).Check(testkit.Rows("4", "5", "6")) + } // Test invalid value. tk.Session().GetSessionVars().AutoIncrementIncrement = -1 @@ -711,27 +720,19 @@ func TestAlterTableAutoIDCache(t *testing.T) { require.NoError(t, err2) tk.MustExec("alter table t_473 auto_id_cache = 100") - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows( + fmt.Sprintf("test t_473 id %d _TIDB_ROWID", val), + "test t_473 id 1 AUTO_INCREMENT", + )) tk.MustExec("insert into t_473 values ()") tk.MustQuery("select * from t_473").Check(testkit.Rows("1", fmt.Sprintf("%d", val))) - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+100))) - - // Note that auto_id_cache=1 use a different implementation. - tk.MustExec("alter table t_473 auto_id_cache = 1") - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+100))) - tk.MustExec("insert into t_473 values ()") - tk.MustQuery("select * from t_473").Check(testkit.Rows("1", fmt.Sprintf("%d", val), fmt.Sprintf("%d", val+100))) - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+101))) - - // alter table from auto_id_cache=1 to default will discard the IDs cached by the autoid service. - // This is because they are two component and TiDB can't tell the autoid service to "save position and exit". - tk.MustExec("alter table t_473 auto_id_cache = 20000") - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+4100))) - - tk.MustExec("insert into t_473 values ()") - tk.MustQuery("select * from t_473").Check(testkit.Rows("1", - fmt.Sprintf("%d", val), - fmt.Sprintf("%d", val+100), - fmt.Sprintf("%d", val+4100))) - tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows(fmt.Sprintf("test t_473 id %d AUTO_INCREMENT", val+24100))) + tk.MustQuery("show table t_473 next_row_id").Check(testkit.Rows( + fmt.Sprintf("test t_473 id %d _TIDB_ROWID", val+100), + "test t_473 id 1 AUTO_INCREMENT", + )) + + // Note that auto_id_cache=1 use a different implementation, switch between them is not allowed. + // TODO: relax this restriction and update the test case. + _, err = tk.Exec("alter table t_473 auto_id_cache = 1") + require.Error(t, err) } diff --git a/executor/executor.go b/executor/executor.go index 6aac622b223a4..713a507eef59a 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -399,7 +399,7 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { tblMeta := tbl.Meta() allocators := tbl.Allocators(e.ctx) - for _, alloc := range allocators { + for _, alloc := range allocators.Allocs { nextGlobalID, err := alloc.NextGlobalAutoID() if err != nil { return err @@ -407,7 +407,16 @@ func (e *ShowNextRowIDExec) Next(ctx context.Context, req *chunk.Chunk) error { var colName, idType string switch alloc.GetType() { - case autoid.RowIDAllocType, autoid.AutoIncrementType: + case autoid.RowIDAllocType: + idType = "_TIDB_ROWID" + if tblMeta.PKIsHandle { + if col := tblMeta.GetAutoIncrementColInfo(); col != nil { + colName = col.Name.O + } + } else { + colName = model.ExtraHandleName.O + } + case autoid.AutoIncrementType: idType = "AUTO_INCREMENT" if tblMeta.PKIsHandle { if col := tblMeta.GetAutoIncrementColInfo(); col != nil { diff --git a/executor/executor_test.go b/executor/executor_test.go index 5e8bb71daca6a..c73b3a3df7abd 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -1922,7 +1922,7 @@ func TestCheckIndex(t *testing.T) { tbInfo := tbl.Meta() alloc := autoid.NewAllocator(store, dbInfo.ID, tbInfo.ID, false, autoid.RowIDAllocType) - tb, err := tables.TableFromMeta(autoid.NewAllocators(alloc), tbInfo) + tb, err := tables.TableFromMeta(autoid.NewAllocators(false, alloc), tbInfo) require.NoError(t, err) _, err = se.Execute(context.Background(), "admin check index t c") diff --git a/executor/infoschema_reader.go b/executor/infoschema_reader.go index 1faf10d490856..472220bb2dcc6 100644 --- a/executor/infoschema_reader.go +++ b/executor/infoschema_reader.go @@ -386,7 +386,7 @@ func getAutoIncrementID(ctx sessionctx.Context, schema *model.DBInfo, tblInfo *m if err != nil { return 0, err } - return tbl.Allocators(ctx).Get(autoid.RowIDAllocType).Base() + 1, nil + return tbl.Allocators(ctx).Get(autoid.AutoIncrementType).Base() + 1, nil } func hasPriv(ctx sessionctx.Context, priv mysql.PrivilegeType) bool { diff --git a/executor/insert_common.go b/executor/insert_common.go index 5bb7feb2441da..d0146d9c3922a 100644 --- a/executor/insert_common.go +++ b/executor/insert_common.go @@ -778,7 +778,8 @@ func (e *InsertValues) lazyAdjustAutoIncrementDatum(ctx context.Context, rows [] } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.Allocators(e.ctx).Get(autoid.RowIDAllocType).Rebase(ctx, recordID, true) + alloc := e.Table.Allocators(e.ctx).Get(autoid.AutoIncrementType) + err = alloc.Rebase(ctx, recordID, true) if err != nil { return nil, err } @@ -871,7 +872,7 @@ func (e *InsertValues) adjustAutoIncrementDatum(ctx context.Context, d types.Dat } // Use the value if it's not null and not 0. if recordID != 0 { - err = e.Table.Allocators(e.ctx).Get(autoid.RowIDAllocType).Rebase(ctx, recordID, true) + err = e.Table.Allocators(e.ctx).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true) if err != nil { return types.Datum{}, err } diff --git a/executor/issuetest/executor_issue_test.go b/executor/issuetest/executor_issue_test.go index 78de8f0ef37d6..643339dd157d0 100644 --- a/executor/issuetest/executor_issue_test.go +++ b/executor/issuetest/executor_issue_test.go @@ -1240,3 +1240,46 @@ func TestIssue33214(t *testing.T) { } } } + +func TestIssue982(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t;") + tk.MustExec("create table t (c int auto_increment, key(c)) auto_id_cache 1;") + tk.MustExec("insert into t values();") + tk.MustExec("insert into t values();") + tk.MustQuery("select * from t;").Check(testkit.Rows("1", "2")) +} + +func TestIssue24627(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + for _, sql := range []string{ + "create table test(id float primary key clustered AUTO_INCREMENT, col1 int);", + "create table test(id float primary key nonclustered AUTO_INCREMENT, col1 int) AUTO_ID_CACHE 1;", + } { + tk.MustExec("drop table if exists test;") + tk.MustExec(sql) + tk.MustExec("replace into test(col1) values(1);") + tk.MustExec("replace into test(col1) values(2);") + tk.MustQuery("select * from test;").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("drop table test") + } + + for _, sql := range []string{ + "create table test2(id double primary key clustered AUTO_INCREMENT, col1 int);", + "create table test2(id double primary key nonclustered AUTO_INCREMENT, col1 int) AUTO_ID_CACHE 1;", + } { + tk.MustExec(sql) + tk.MustExec("replace into test2(col1) values(1);") + tk.MustExec("insert into test2(col1) values(1);") + tk.MustExec("replace into test2(col1) values(1);") + tk.MustExec("insert into test2(col1) values(1);") + tk.MustExec("replace into test2(col1) values(1);") + tk.MustExec("replace into test2(col1) values(1);") + tk.MustQuery("select * from test2").Check(testkit.Rows("1 1", "2 1", "3 1", "4 1", "5 1", "6 1")) + tk.MustExec("drop table test2") + } +} diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 7f16fd68da5e7..bf78e40e7ac3d 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -773,43 +773,45 @@ func HelperTestAdminShowNextID(t *testing.T, store kv.Storage, str string) { tk.MustExec("create table t(id int, c int)") // Start handle is 1. r := tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 1 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 1 _TIDB_ROWID")) // Row ID is step + 1. tk.MustExec("insert into t values(1, 1)") r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 11 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 11 _TIDB_ROWID")) // Row ID is original + step. for i := 0; i < int(step); i++ { tk.MustExec("insert into t values(10000, 1)") } r = tk.MustQuery(str + " t next_row_id") - r.Check(testkit.Rows("test t _tidb_rowid 21 AUTO_INCREMENT")) + r.Check(testkit.Rows("test t _tidb_rowid 21 _TIDB_ROWID")) tk.MustExec("drop table t") // test for a table with the primary key tk.MustExec("create table tt(id int primary key auto_increment, c int)") // Start handle is 1. r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test tt id 1 AUTO_INCREMENT")) + r.Check(testkit.Rows("test tt id 1 _TIDB_ROWID", "test tt id 1 AUTO_INCREMENT")) // After rebasing auto ID, row ID is 20 + step + 1. tk.MustExec("insert into tt values(20, 1)") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test tt id 31 AUTO_INCREMENT")) + r.Check(testkit.Rows("test tt id 31 _TIDB_ROWID", "test tt id 1 AUTO_INCREMENT")) // test for renaming the table tk.MustExec("drop database if exists test1") tk.MustExec("create database test1") tk.MustExec("rename table test.tt to test1.tt") tk.MustExec("use test1") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test1 tt id 31 AUTO_INCREMENT")) + r.Check(testkit.Rows("test1 tt id 31 _TIDB_ROWID", "test1 tt id 1 AUTO_INCREMENT")) tk.MustExec("insert test1.tt values ()") r = tk.MustQuery(str + " tt next_row_id") - r.Check(testkit.Rows("test1 tt id 41 AUTO_INCREMENT")) + r.Check(testkit.Rows("test1 tt id 41 _TIDB_ROWID", "test1 tt id 1 AUTO_INCREMENT")) tk.MustExec("drop table tt") tk.MustExec("drop table if exists t;") tk.MustExec("create table t (a int auto_increment primary key nonclustered, b int);") - tk.MustQuery("show table t next_row_id;").Check(testkit.Rows("test1 t _tidb_rowid 1 AUTO_INCREMENT")) + tk.MustQuery("show table t next_row_id;").Check(testkit.Rows( + "test1 t _tidb_rowid 1 _TIDB_ROWID", + "test1 t _tidb_rowid 1 AUTO_INCREMENT")) tk.MustExec("set @@allow_auto_random_explicit_insert = true") @@ -830,19 +832,19 @@ func HelperTestAdminShowNextID(t *testing.T, store kv.Storage, str string) { // Test for a sequence. tk.MustExec("create sequence seq1 start 15 cache 57") r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 15 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 15 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("15")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select nextval(seq1)") r.Check(testkit.Rows("16")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 72 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 72 SEQUENCE")) r = tk.MustQuery("select setval(seq1, 96)") r.Check(testkit.Rows("96")) r = tk.MustQuery(str + " seq1 next_row_id") - r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 AUTO_INCREMENT", "test1 seq1 97 SEQUENCE")) + r.Check(testkit.Rows("test1 seq1 _tidb_rowid 1 _TIDB_ROWID", "test1 seq1 97 SEQUENCE")) } func TestNoHistoryWhenDisableRetry(t *testing.T) { diff --git a/executor/write.go b/executor/write.go index 4c36884cc570d..363bb097fd02c 100644 --- a/executor/write.go +++ b/executor/write.go @@ -109,7 +109,7 @@ func updateRecord(ctx context.Context, sctx sessionctx.Context, h kv.Handle, old if err != nil { return false, err } - if err = t.Allocators(sctx).Get(autoid.RowIDAllocType).Rebase(ctx, recordID, true); err != nil { + if err = t.Allocators(sctx).Get(autoid.AutoIncrementType).Rebase(ctx, recordID, true); err != nil { return false, err } } diff --git a/infoschema/builder.go b/infoschema/builder.go index 8ff50a8b50435..67153fcc5f3bd 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -693,18 +693,23 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i ConvertCharsetCollateToLowerCaseIfNeed(tblInfo) ConvertOldVersionUTF8ToUTF8MB4IfNeed(tblInfo) - if len(allocs) == 0 { + if len(allocs.Allocs) == 0 { allocs = autoid.NewAllocatorsFromTblInfo(b.store, dbInfo.ID, tblInfo) } else { tblVer := autoid.AllocOptionTableInfoVersion(tblInfo.Version) switch tp { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: idCacheOpt := autoid.CustomAutoIncCacheOption(tblInfo.AutoIdCache) - newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), autoid.RowIDAllocType, tblVer, idCacheOpt) - allocs = append(allocs, newAlloc) + // If the allocator type might be AutoIncrementType, create both AutoIncrementType + // and RowIDAllocType allocator for it. Because auto id and row id could share the same allocator. + // Allocate auto id may route to allocate row id, if row id allocator is nil, the program panic! + for _, tp := range [2]autoid.AllocatorType{autoid.AutoIncrementType, autoid.RowIDAllocType} { + newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), tp, tblVer, idCacheOpt) + allocs = allocs.Append(newAlloc) + } case model.ActionRebaseAutoRandomBase: newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) - allocs = append(allocs, newAlloc) + allocs = allocs.Append(newAlloc) case model.ActionModifyColumn: // Change column attribute from auto_increment to auto_random. if tblInfo.ContainsAutoRandomBits() && allocs.Get(autoid.AutoRandomType) == nil { @@ -713,7 +718,7 @@ func (b *Builder) applyCreateTable(m *meta.Meta, dbInfo *model.DBInfo, tableID i return a.GetType() != autoid.AutoIncrementType && a.GetType() != autoid.RowIDAllocType }) newAlloc := autoid.NewAllocator(b.store, dbInfo.ID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), autoid.AutoRandomType, tblVer) - allocs = append(allocs, newAlloc) + allocs = allocs.Append(newAlloc) } } } diff --git a/infoschema/infoschema.go b/infoschema/infoschema.go index 040dbd239eef6..11c9bd8b0e7af 100644 --- a/infoschema/infoschema.go +++ b/infoschema/infoschema.go @@ -270,7 +270,7 @@ func (is *infoSchema) TableByID(id int64) (val table.Table, ok bool) { func (is *infoSchema) AllocByID(id int64) (autoid.Allocators, bool) { tbl, ok := is.TableByID(id) if !ok { - return nil, false + return autoid.Allocators{}, false } return tbl.Allocators(nil), true } diff --git a/infoschema/tables.go b/infoschema/tables.go index 8df6cab9cb84c..d76d8b8be60b0 100644 --- a/infoschema/tables.go +++ b/infoschema/tables.go @@ -2123,7 +2123,7 @@ func (it *infoschemaTable) UpdateRecord(gctx context.Context, ctx sessionctx.Con // Allocators implements table.Table Allocators interface. func (it *infoschemaTable) Allocators(_ sessionctx.Context) autoid.Allocators { - return nil + return autoid.Allocators{} } // Meta implements table.Table Meta interface. @@ -2206,7 +2206,7 @@ func (vt *VirtualTable) UpdateRecord(ctx context.Context, sctx sessionctx.Contex // Allocators implements table.Table Allocators interface. func (vt *VirtualTable) Allocators(_ sessionctx.Context) autoid.Allocators { - return nil + return autoid.Allocators{} } // Meta implements table.Table Meta interface. diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index def3245bb2da3..2a181a088d802 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -205,16 +205,36 @@ type Allocator interface { } // Allocators represents a set of `Allocator`s. -type Allocators []Allocator +type Allocators struct { + SepAutoInc bool + Allocs []Allocator +} // NewAllocators packs multiple `Allocator`s into Allocators. -func NewAllocators(allocators ...Allocator) Allocators { - return allocators +func NewAllocators(sepAutoInc bool, allocators ...Allocator) Allocators { + return Allocators{ + SepAutoInc: sepAutoInc, + Allocs: allocators, + } +} + +// Append add an allocator to the allocators. +func (all Allocators) Append(a Allocator) Allocators { + return Allocators{ + SepAutoInc: all.SepAutoInc, + Allocs: append(all.Allocs, a), + } } // Get returns the Allocator according to the AllocatorType. func (all Allocators) Get(allocType AllocatorType) Allocator { - for _, a := range all { + if !all.SepAutoInc { + if allocType == AutoIncrementType { + allocType = RowIDAllocType + } + } + + for _, a := range all.Allocs { if a.GetType() == allocType { return a } @@ -224,13 +244,16 @@ func (all Allocators) Get(allocType AllocatorType) Allocator { // Filter filters all the allocators that match pred. func (all Allocators) Filter(pred func(Allocator) bool) Allocators { - var ret Allocators - for _, a := range all { + var ret []Allocator + for _, a := range all.Allocs { if pred(a) { ret = append(ret, a) } } - return ret + return Allocators{ + SepAutoInc: all.SepAutoInc, + Allocs: ret, + } } type allocator struct { @@ -593,7 +616,7 @@ func NewAllocator(store kv.Storage, dbID, tbID int64, isUnsigned bool, } // Use the MySQL compatible AUTO_INCREMENT mode. - if allocType == RowIDAllocType && alloc.customStep && alloc.step == 1 { + if allocType == AutoIncrementType && alloc.customStep && alloc.step == 1 && alloc.tbVersion >= model.TableInfoVersion5 { alloc1 := newSinglePointAlloc(store, dbID, tbID, isUnsigned) if alloc1 != nil { return alloc1 @@ -630,6 +653,10 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), RowIDAllocType, idCacheOpt, tblVer) allocs = append(allocs, alloc) } + if hasAutoIncID { + alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoIncColUnsigned(), AutoIncrementType, idCacheOpt, tblVer) + allocs = append(allocs, alloc) + } hasAutoRandID := tblInfo.ContainsAutoRandomBits() if hasAutoRandID { alloc := NewAllocator(store, dbID, tblInfo.ID, tblInfo.IsAutoRandomBitColUnsigned(), AutoRandomType, idCacheOpt, tblVer) @@ -638,7 +665,7 @@ func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.T if tblInfo.IsSequence() { allocs = append(allocs, NewSequenceAllocator(store, dbID, tblInfo.ID, tblInfo.Sequence)) } - return NewAllocators(allocs...) + return NewAllocators(tblInfo.SepAutoInc(), allocs...) } // Alloc implements autoid.Allocator Alloc interface. @@ -839,7 +866,7 @@ func (alloc *allocator) alloc4Signed(ctx context.Context, n uint64, increment, o var newBase, newEnd int64 startTime := time.Now() nextStep := alloc.step - if !alloc.customStep { + if !alloc.customStep && alloc.end > 0 { // Although it may skip a segment here, we still think it is consumed. consumeDur := startTime.Sub(alloc.lastAllocTime) nextStep = NextStep(alloc.step, consumeDur) diff --git a/meta/autoid/autoid_service.go b/meta/autoid/autoid_service.go index 2942f3281b769..e524397e0041f 100644 --- a/meta/autoid/autoid_service.go +++ b/meta/autoid/autoid_service.go @@ -236,5 +236,5 @@ func (sp *singlePointAlloc) NextGlobalAutoID() (int64, error) { } func (*singlePointAlloc) GetType() AllocatorType { - return RowIDAllocType + return AutoIncrementType } diff --git a/meta/meta_autoid.go b/meta/meta_autoid.go index 18d384b2b25a7..5763aa268051a 100644 --- a/meta/meta_autoid.go +++ b/meta/meta_autoid.go @@ -102,7 +102,7 @@ type autoIDAccessors struct { access autoIDAccessor } -const sepAutoIncVer = model.TableInfoVersion4 + 1 +const sepAutoIncVer = model.TableInfoVersion5 // Get implements the interface AutoIDAccessors. func (a *autoIDAccessors) Get() (autoIDs AutoIDGroup, err error) { diff --git a/parser/model/model.go b/parser/model/model.go index e38e5b368f4a1..ba7c46bcd6333 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -446,14 +446,16 @@ const ( // However, the convert is missed in some scenarios before v2.1.9, so for all those tables prior to TableInfoVersion3, their // charsets / collations will be converted to lower-case while loading from the storage. TableInfoVersion3 = uint16(3) - // TableInfoVersion4 indicates that the auto_increment allocator in TiDB has been separated from - // _tidb_rowid allocator. This version is introduced to preserve the compatibility of old tables: - // the tables with version < TableInfoVersion4 still use a single allocator for auto_increment and _tidb_rowid. - // Also see https://github.com/pingcap/tidb/issues/982. + // TableInfoVersion4 is not used. TableInfoVersion4 = uint16(4) + // TableInfoVersion5 indicates that the auto_increment allocator in TiDB has been separated from + // _tidb_rowid allocator when AUTO_ID_CACHE is 1. This version is introduced to preserve the compatibility of old tables: + // the tables with version <= TableInfoVersion4 still use a single allocator for auto_increment and _tidb_rowid. + // Also see https://github.com/pingcap/tidb/issues/982. + TableInfoVersion5 = uint16(5) // CurrLatestTableInfoVersion means the latest table info in the current TiDB. - CurrLatestTableInfoVersion = TableInfoVersion4 + CurrLatestTableInfoVersion = TableInfoVersion5 ) // ExtraHandleName is the name of ExtraHandle Column. @@ -552,6 +554,11 @@ type TableInfo struct { TTLInfo *TTLInfo `json:"ttl_info"` } +// SepAutoInc decides whether _rowid and auto_increment id use separate allocator. +func (t *TableInfo) SepAutoInc() bool { + return t.Version >= TableInfoVersion5 && t.AutoIdCache == 1 +} + // TableCacheStatusType is the type of the table cache status type TableCacheStatusType int diff --git a/session/BUILD.bazel b/session/BUILD.bazel index 2a79502bf985e..d0d68a9142035 100644 --- a/session/BUILD.bazel +++ b/session/BUILD.bazel @@ -136,6 +136,7 @@ go_test( "//expression", "//kv", "//meta", + "//meta/autoid", "//parser/ast", "//parser/auth", "//parser/model", diff --git a/session/schema_amender_test.go b/session/schema_amender_test.go index e82faba8e6e56..565b4861a9400 100644 --- a/session/schema_amender_test.go +++ b/session/schema_amender_test.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/kvproto/pkg/kvrpcpb" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/planner/core" @@ -276,7 +277,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) { initTblColIdxID(oldTblMeta) // Indices[0] does not exist at the start. oldTblMeta.Indices = oldTblMeta.Indices[1:] - oldTbInfo, err := table.TableFromMeta(nil, oldTblMeta) + oldTbInfo, err := table.TableFromMeta(autoid.NewAllocators(false), oldTblMeta) require.NoError(t, err) oldTblMeta.Indices[0].State = startState oldTblMeta.Indices[2].State = endState @@ -296,7 +297,7 @@ func TestAmendCollectAndGenMutations(t *testing.T) { // The last index "c_d_e_str_prefix is dropped. newTblMeta.Indices = newTblMeta.Indices[:len(newTblMeta.Indices)-1] newTblMeta.Indices[0].Unique = false - newTblInfo, err := table.TableFromMeta(nil, newTblMeta) + newTblInfo, err := table.TableFromMeta(autoid.NewAllocators(false), newTblMeta) require.NoError(t, err) newTblMeta.Indices[0].State = endState // Indices[1] is newly created. diff --git a/table/table.go b/table/table.go index 6aef5ed4497e6..813131df90896 100644 --- a/table/table.go +++ b/table/table.go @@ -206,7 +206,8 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte } increment := sctx.GetSessionVars().AutoIncrementIncrement offset := sctx.GetSessionVars().AutoIncrementOffset - _, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(1), int64(increment), int64(offset)) + alloc := t.Allocators(sctx).Get(autoid.AutoIncrementType) + _, max, err := alloc.Alloc(ctx, uint64(1), int64(increment), int64(offset)) if err != nil { return 0, err } @@ -218,7 +219,8 @@ func AllocAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Conte func AllocBatchAutoIncrementValue(ctx context.Context, t Table, sctx sessionctx.Context, N int) (firstID int64, increment int64, err error) { increment = int64(sctx.GetSessionVars().AutoIncrementIncrement) offset := int64(sctx.GetSessionVars().AutoIncrementOffset) - min, max, err := t.Allocators(sctx).Get(autoid.RowIDAllocType).Alloc(ctx, uint64(N), increment, offset) + alloc := t.Allocators(sctx).Get(autoid.AutoIncrementType) + min, max, err := alloc.Alloc(ctx, uint64(N), increment, offset) if err != nil { return min, max, err } diff --git a/table/tables/tables.go b/table/tables/tables.go index 825f05d7ffa2e..9a5eaae0d6096 100644 --- a/table/tables/tables.go +++ b/table/tables/tables.go @@ -85,7 +85,7 @@ func MockTableFromMeta(tblInfo *model.TableInfo) table.Table { } var t TableCommon - initTableCommon(&t, tblInfo, tblInfo.ID, columns, nil) + initTableCommon(&t, tblInfo, tblInfo.ID, columns, autoid.NewAllocators(false)) if tblInfo.TableCacheStatusType != model.TableCacheStatusDisable { ret, err := newCachedTable(&t) if err != nil { @@ -1547,7 +1547,7 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { // Use an independent allocator for global temporary tables. if t.meta.TempTableType == model.TempTableGlobal { if alloc := ctx.GetSessionVars().GetTemporaryTable(t.meta).GetAutoIDAllocator(); alloc != nil { - return autoid.Allocators{alloc} + return autoid.NewAllocators(false, alloc) } // If the session is not in a txn, for example, in "show create table", use the original allocator. // Otherwise the would be a nil pointer dereference. @@ -1557,8 +1557,9 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { // Replace the row id allocator with the one in session variables. sessAlloc := ctx.GetSessionVars().IDAllocator - retAllocs := make([]autoid.Allocator, 0, len(t.allocs)) - copy(retAllocs, t.allocs) + allocs := t.allocs.Allocs + retAllocs := make([]autoid.Allocator, 0, len(allocs)) + copy(retAllocs, allocs) overwritten := false for i, a := range retAllocs { @@ -1571,7 +1572,7 @@ func (t *TableCommon) Allocators(ctx sessionctx.Context) autoid.Allocators { if !overwritten { retAllocs = append(retAllocs, sessAlloc) } - return retAllocs + return autoid.NewAllocators(t.allocs.SepAutoInc, retAllocs...) } // Type implements table.Table Type interface. @@ -1925,7 +1926,7 @@ func maxIndexLen(idxA, idxB *model.IndexColumn) *model.IndexColumn { } func getSequenceAllocator(allocs autoid.Allocators) (autoid.Allocator, error) { - for _, alloc := range allocs { + for _, alloc := range allocs.Allocs { if alloc.GetType() == autoid.SequenceType { return alloc, nil } diff --git a/table/tables/tables_test.go b/table/tables/tables_test.go index d659f918a8168..661770b868383 100644 --- a/table/tables/tables_test.go +++ b/table/tables/tables_test.go @@ -378,18 +378,18 @@ func TestTableFromMeta(t *testing.T) { // For test coverage tbInfo.Columns[0].GeneratedExprString = "a" - _, err = tables.TableFromMeta(nil, tbInfo) + _, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.NoError(t, err) tbInfo.Columns[0].GeneratedExprString = "test" - _, err = tables.TableFromMeta(nil, tbInfo) + _, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.Error(t, err) tbInfo.Columns[0].State = model.StateNone - tb, err = tables.TableFromMeta(nil, tbInfo) + tb, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.Nil(t, tb) require.Error(t, err) tbInfo.State = model.StateNone - tb, err = tables.TableFromMeta(nil, tbInfo) + tb, err = tables.TableFromMeta(autoid.NewAllocators(false), tbInfo) require.Nil(t, tb) require.Error(t, err) diff --git a/table/temptable/BUILD.bazel b/table/temptable/BUILD.bazel index 30c41bd1c55b3..8487a26533b51 100644 --- a/table/temptable/BUILD.bazel +++ b/table/temptable/BUILD.bazel @@ -41,6 +41,7 @@ go_test( deps = [ "//infoschema", "//kv", + "//meta/autoid", "//parser/model", "//parser/mysql", "//sessionctx", diff --git a/table/temptable/ddl.go b/table/temptable/ddl.go index ccad2b7b0214c..d464cb3c48618 100644 --- a/table/temptable/ddl.go +++ b/table/temptable/ddl.go @@ -182,7 +182,7 @@ func newTemporaryTableFromTableInfo(sctx sessionctx.Context, tbInfo *model.Table if alloc != nil { allocs = append(allocs, alloc) } - return tables.TableFromMeta(allocs, tbInfo) + return tables.TableFromMeta(autoid.NewAllocators(false, allocs...), tbInfo) } // GetTemporaryTableDDL gets the temptable.TemporaryTableDDL from session context diff --git a/table/temptable/main_test.go b/table/temptable/main_test.go index 7dbc5e86d37d9..8c5c4f557e1ae 100644 --- a/table/temptable/main_test.go +++ b/table/temptable/main_test.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/infoschema" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/meta/autoid" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" "github.com/pingcap/tidb/table" @@ -86,7 +87,7 @@ func (is *mockedInfoSchema) TableByID(tblID int64) (table.Table, bool) { State: model.StatePublic, } - tbl, err := table.TableFromMeta(nil, tblInfo) + tbl, err := table.TableFromMeta(autoid.NewAllocators(false), tblInfo) require.NoError(is.t, err) return tbl, true diff --git a/telemetry/data_feature_usage_test.go b/telemetry/data_feature_usage_test.go index bfb6922db7d30..ebe93d6dbaa52 100644 --- a/telemetry/data_feature_usage_test.go +++ b/telemetry/data_feature_usage_test.go @@ -143,7 +143,7 @@ func TestAutoIDNoCache(t *testing.T) { usage, err = telemetry.GetFeatureUsage(tk.Session()) require.NoError(t, err) require.True(t, usage.AutoIDNoCache) - tk.MustExec("alter table tele_autoid auto_id_cache=0") + tk.MustExec("drop table tele_autoid") usage, err = telemetry.GetFeatureUsage(tk.Session()) require.NoError(t, err) require.False(t, usage.AutoIDNoCache)