From 19a0c9239721cf56104da6b0ab4645fb36302bc7 Mon Sep 17 00:00:00 2001 From: Mattias Jonsson Date: Fri, 8 Dec 2023 10:45:19 +0100 Subject: [PATCH] This is an automated cherry-pick of #47892 Signed-off-by: ti-chi-bot --- ddl/table.go | 23 +- executor/seqtest/seq_executor_test.go | 9 + infoschema/builder.go | 9 + meta/autoid/autoid.go | 4 +- meta/meta_autoid.go | 42 +- meta/meta_test.go | 62 +- parser/model/model.go | 15 +- pkg/ddl/db_rename_test.go | 482 ++++++ pkg/ddl/schema_test.go | 451 ++++++ pkg/executor/test/autoidtest/autoid_test.go | 569 ++++++++ pkg/executor/test/ddl/ddl_test.go | 1021 +++++++++++++ pkg/meta/BUILD.bazel | 46 + .../r/ddl/db_integration.result | 1293 +++++++++++++++++ .../integrationtest/r/executor/autoid.result | 739 ++++++++++ tests/integrationtest/t/executor/autoid.test | 485 +++++++ 15 files changed, 5201 insertions(+), 49 deletions(-) create mode 100644 pkg/ddl/db_rename_test.go create mode 100644 pkg/ddl/schema_test.go create mode 100644 pkg/executor/test/autoidtest/autoid_test.go create mode 100644 pkg/executor/test/ddl/ddl_test.go create mode 100644 pkg/meta/BUILD.bazel create mode 100644 tests/integrationtest/r/ddl/db_integration.result create mode 100644 tests/integrationtest/r/executor/autoid.result create mode 100644 tests/integrationtest/t/executor/autoid.test diff --git a/ddl/table.go b/ddl/table.go index a60e5d9a76b20..eed2a46c6f898 100644 --- a/ddl/table.go +++ b/ddl/table.go @@ -798,7 +798,14 @@ func onRenameTables(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error return ver, errors.Trace(err) } +<<<<<<< HEAD:ddl/table.go var tblInfos = make([]*model.TableInfo, 0, len(tableNames)) +======= + if job.SchemaState == model.StatePublic { + return finishJobRenameTables(d, t, job, tableNames, tableIDs, newSchemaIDs) + } + +>>>>>>> a3e2ddb5864 (*: Keep the auto id allocator for single table renames (#47892)):pkg/ddl/table.go var err error for i, oldSchemaID := range oldSchemaIDs { job.TableID = tableIDs[i] @@ -806,7 +813,6 @@ func onRenameTables(d *ddlCtx, t *meta.Meta, job *model.Job) (ver int64, _ error if err != nil { return ver, errors.Trace(err) } - tblInfos = append(tblInfos, tblInfo) } ver, err = updateSchemaVersion(t, job) @@ -845,6 +851,18 @@ func checkAndRenameTables(t *meta.Meta, job *model.Job, oldSchemaID, newSchemaID return ver, tblInfo, errors.Wrapf(err, "failed to get old label rules from PD") } + if tblInfo.AutoIDSchemaID == 0 && newSchemaID != oldSchemaID { + // The auto id is referenced by a schema id + table id + // Table ID is not changed between renames, but schema id can change. + // To allow concurrent use of the auto id during rename, keep the auto id + // by always reference it with the schema id it was originally created in. + tblInfo.AutoIDSchemaID = oldSchemaID + } + if newSchemaID == tblInfo.AutoIDSchemaID { + // Back to the original schema id, no longer needed. + tblInfo.AutoIDSchemaID = 0 + } + tblInfo.Name = *tableName err = t.CreateTableOrView(newSchemaID, tblInfo) if err != nil { @@ -852,6 +870,7 @@ func checkAndRenameTables(t *meta.Meta, job *model.Job, oldSchemaID, newSchemaID return ver, tblInfo, errors.Trace(err) } +<<<<<<< HEAD:ddl/table.go if newSchemaID != oldSchemaID { oldDBID := tblInfo.GetDBID(oldSchemaID) err := meta.BackupAndRestoreAutoIDs(t, oldDBID, tblInfo.ID, newSchemaID, tblInfo.ID) @@ -864,6 +883,8 @@ func checkAndRenameTables(t *meta.Meta, job *model.Job, oldSchemaID, newSchemaID tblInfo.OldSchemaID = 0 } +======= +>>>>>>> a3e2ddb5864 (*: Keep the auto id allocator for single table renames (#47892)):pkg/ddl/table.go err = updateLabelRules(job, tblInfo, oldRules, tableRuleID, partRuleIDs, oldRuleIDs, tblInfo.ID) if err != nil { job.State = model.JobStateCancelled diff --git a/executor/seqtest/seq_executor_test.go b/executor/seqtest/seq_executor_test.go index 8019843b50f84..c539c655fda3c 100644 --- a/executor/seqtest/seq_executor_test.go +++ b/executor/seqtest/seq_executor_test.go @@ -809,10 +809,19 @@ func HelperTestAdminShowNextID(t *testing.T, store kv.Storage, str string) { tk.MustExec("rename table test.tt to test1.tt") tk.MustExec("use test1") r = tk.MustQuery(str + " tt next_row_id") +<<<<<<< HEAD:executor/seqtest/seq_executor_test.go r.Check(testkit.Rows("test1 tt id 31 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 31 _TIDB_ROWID", "test1 tt id 1 AUTO_INCREMENT")) + tk.MustQuery(`select * from tt`).Sort().Check(testkit.Rows("20 1")) + tk.MustExec("insert test1.tt values ()") + r = tk.MustQuery(str + " tt next_row_id") + r.Check(testkit.Rows("test1 tt id 31 _TIDB_ROWID", "test1 tt id 1 AUTO_INCREMENT")) + tk.MustQuery(`select * from tt`).Sort().Check(testkit.Rows("20 1", "21 ")) +>>>>>>> a3e2ddb5864 (*: Keep the auto id allocator for single table renames (#47892)):pkg/executor/test/seqtest/seq_executor_test.go tk.MustExec("drop table tt") tk.MustExec("set @@allow_auto_random_explicit_insert = true") diff --git a/infoschema/builder.go b/infoschema/builder.go index 12b8807f84385..89f5fe975c467 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -213,8 +213,14 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 // We try to reuse the old allocator, so the cached auto ID can be reused. var allocs autoid.Allocators if tableIDIsValid(oldTableID) { +<<<<<<< HEAD:infoschema/builder.go if oldTableID == newTableID && (diff.Type != model.ActionRenameTable && diff.Type != model.ActionRenameTables) && diff.Type != model.ActionExchangeTablePartition && +======= + if oldTableID == newTableID && + // For rename table, keep the old alloc. + +>>>>>>> a3e2ddb5864 (*: Keep the auto id allocator for single table renames (#47892)):pkg/infoschema/builder.go // For repairing table in TiDB cluster, given 2 normal node and 1 repair node. // For normal node's information schema, repaired table is existed. // For repair node's information schema, repaired table is filtered (couldn't find it in `is`). @@ -222,6 +228,9 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 diff.Type != model.ActionRepairTable && // Alter sequence will change the sequence info in the allocator, so the old allocator is not valid any more. diff.Type != model.ActionAlterSequence { + // TODO: Check how this would work with ADD/REMOVE Partitioning, + // which may have AutoID not connected to tableID + // TODO: can there be _tidb_rowid AutoID per partition? oldAllocs, _ := b.is.AllocByID(oldTableID) allocs = filterAllocators(diff, oldAllocs) } diff --git a/meta/autoid/autoid.go b/meta/autoid/autoid.go index 8ee99f5939a0e..81057441ff6bb 100644 --- a/meta/autoid/autoid.go +++ b/meta/autoid/autoid.go @@ -198,7 +198,7 @@ type allocator struct { base int64 end int64 store kv.Storage - // dbID is current database's ID. + // dbID is database ID where it was created. dbID int64 tbID int64 tbVersion uint16 @@ -525,7 +525,7 @@ func NewSequenceAllocator(store kv.Storage, dbID, tbID int64, info *model.Sequen // NewAllocatorsFromTblInfo creates an array of allocators of different types with the information of model.TableInfo. func NewAllocatorsFromTblInfo(store kv.Storage, schemaID int64, tblInfo *model.TableInfo) Allocators { var allocs []Allocator - dbID := tblInfo.GetDBID(schemaID) + dbID := tblInfo.GetAutoIDSchemaID(schemaID) idCacheOpt := CustomAutoIncCacheOption(tblInfo.AutoIdCache) tblVer := AllocOptionTableInfoVersion(tblInfo.Version) diff --git a/meta/meta_autoid.go b/meta/meta_autoid.go index 18d384b2b25a7..c83e17c8026ad 100644 --- a/meta/meta_autoid.go +++ b/meta/meta_autoid.go @@ -54,17 +54,17 @@ func (a *autoIDAccessor) Put(val int64) error { // Inc implements the interface AutoIDAccessor. func (a *autoIDAccessor) Inc(step int64) (int64, error) { m := a.m + // Note that the databaseID may not match the current table, + // it may come from the original schema id the table was created + // in, but to allow concurrent use across renames etc. we keep + // the full ID (Schema ID + Table ID) as is. + // Meaning we cannot verify only the schema id. + // And a rename may have happened before the first id is set, + // as well as dropping the original schema. + // So no Schema ID or Table ID verifications can be done. dbKey := m.dbKey(a.databaseID) - if err := m.checkDBExists(dbKey); err != nil { - return 0, errors.Trace(err) - } - // Check if table exists. - tableKey := m.tableKey(a.tableID) - if err := m.checkTableExists(dbKey, tableKey); err != nil { - return 0, errors.Trace(err) - } - - return m.txn.HInc(dbKey, a.idEncodeFn(a.tableID), step) + tblKey := a.idEncodeFn(a.tableID) + return m.txn.HInc(dbKey, tblKey, step) } // Del implements the interface AutoIDAccessor. @@ -192,25 +192,3 @@ type AutoIDGroup struct { IncrementID int64 RandomID int64 } - -// BackupAndRestoreAutoIDs changes the meta key-values to fetch & delete -// all the auto IDs from an old table, and set them to a new table. -func BackupAndRestoreAutoIDs(m *Meta, databaseID, tableID int64, newDatabaseID, newTableID int64) (err error) { - acc := NewAutoIDAccessors(m, databaseID, tableID) - autoIDs, err := acc.Get() - if err != nil { - return errors.Trace(err) - } - overwriteIDs := databaseID == newDatabaseID && tableID == newTableID - if !overwriteIDs { - err = acc.Del() - if err != nil { - return errors.Trace(err) - } - } - err = NewAutoIDAccessors(m, newDatabaseID, newTableID).Put(autoIDs) - if err != nil { - return errors.Trace(err) - } - return nil -} diff --git a/meta/meta_test.go b/meta/meta_test.go index 29441e3e39c52..f18eeec8ebdca 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -102,16 +102,26 @@ func TestPlacementPolicy(t *testing.T) { require.NoError(t, err) } +<<<<<<< HEAD:meta/meta_test.go func TestBackupAndRestoreAutoIDs(t *testing.T) { store, err := mockstore.NewMockStore() require.NoError(t, err) defer func() { err := store.Close() require.NoError(t, err) +======= +func TestResourceGroup(t *testing.T) { + store, err := mockstore.NewMockStore() + require.NoError(t, err) + + defer func() { + require.NoError(t, store.Close()) +>>>>>>> a3e2ddb5864 (*: Keep the auto id allocator for single table renames (#47892)):pkg/meta/meta_test.go }() txn, err := store.Begin() require.NoError(t, err) +<<<<<<< HEAD:meta/meta_test.go m := meta.NewMeta(txn) acc := m.GetAutoIDAccessors(1, 1) require.NoError(t, acc.RowID().Put(100)) @@ -147,6 +157,44 @@ func TestBackupAndRestoreAutoIDs(t *testing.T) { acc2 = m.GetAutoIDAccessors(2, 2) require.Equal(t, mustGet(acc2.RowID()), 100) require.Equal(t, mustGet(acc2.RandomID()), 101) +======= + + // test the independent policy ID allocation. + m := meta.NewMeta(txn) + groups, err := m.ListResourceGroups() + require.NoError(t, err) + require.Equal(t, len(groups), 1) + require.Equal(t, groups[0], meta.DefaultGroupMeta4Test()) + + groupID := int64(2) + checkResourceGroup := func(ru uint64) { + rg, err := m.GetResourceGroup(groupID) + require.NoError(t, err) + require.Equal(t, rg.RURate, ru) + } + + rg := &model.ResourceGroupInfo{ + ID: groupID, + Name: model.NewCIStr("aa"), + ResourceGroupSettings: &model.ResourceGroupSettings{ + RURate: 100, + }, + } + require.NoError(t, m.AddResourceGroup(rg)) + checkResourceGroup(100) + + groups, err = m.ListResourceGroups() + require.NoError(t, err) + require.Equal(t, len(groups), 2) + + rg.RURate = 200 + require.NoError(t, m.UpdateResourceGroup(rg)) + checkResourceGroup(200) + + m.DropResourceGroup(groupID) + _, err = m.GetResourceGroup(groupID) + require.Error(t, err) +>>>>>>> a3e2ddb5864 (*: Keep the auto id allocator for single table renames (#47892)):pkg/meta/meta_test.go } func TestMeta(t *testing.T) { @@ -305,18 +353,20 @@ func TestMeta(t *testing.T) { n, err = m.GetAutoIDAccessors(currentDBID, tid).RowID().Inc(10) require.NoError(t, err) require.Equal(t, int64(10), n) - // Fail to update auto ID. + // Test to update non-existing auto ID. // The table ID doesn't exist. + // We can no longer test for non-existing ids. nonExistentID := int64(1234) _, err = m.GetAutoIDAccessors(currentDBID, nonExistentID).RowID().Inc(10) - require.NotNil(t, err) - require.True(t, meta.ErrTableNotExists.Equal(err)) - // Fail to update auto ID. + require.NoError(t, err) + //require.True(t, meta.ErrTableNotExists.Equal(err)) + // Test to update non-existing auto ID. // The current database ID doesn't exist. + // We can no longer test for non-existing ids. currentDBID = nonExistentID _, err = m.GetAutoIDAccessors(currentDBID, tid).RowID().Inc(10) - require.NotNil(t, err) - require.True(t, meta.ErrDBNotExists.Equal(err)) + require.NoError(t, err) + //require.True(t, meta.ErrDBNotExists.Equal(err)) // Test case for CreateTableAndSetAutoID. tbInfo3 := &model.TableInfo{ ID: 3, diff --git a/parser/model/model.go b/parser/model/model.go index c1569a1f9e7b8..f2c35fe9e4aec 100644 --- a/parser/model/model.go +++ b/parser/model/model.go @@ -311,9 +311,9 @@ type TableInfo struct { // Because auto increment ID has schemaID as prefix, // We need to save original schemaID to keep autoID unchanged // while renaming a table from one database to another. - // TODO: Remove it. - // Now it only uses for compatibility with the old version that already uses this field. - OldSchemaID int64 `json:"old_schema_id,omitempty"` + // Only set if table has been renamed across schemas + // Old name 'old_schema_id' is kept for backwards compatibility + AutoIDSchemaID int64 `json:"old_schema_id,omitempty"` // ShardRowIDBits specify if the implicit row ID is sharded. ShardRowIDBits uint64 @@ -516,11 +516,10 @@ func (t *TableInfo) GetUpdateTime() time.Time { return TSConvert2Time(t.UpdateTS) } -// GetDBID returns the schema ID that is used to create an allocator. -// TODO: Remove it after removing OldSchemaID. -func (t *TableInfo) GetDBID(dbID int64) int64 { - if t.OldSchemaID != 0 { - return t.OldSchemaID +// GetAutoIDSchemaID returns the schema ID that was used to create an allocator. +func (t *TableInfo) GetAutoIDSchemaID(dbID int64) int64 { + if t.AutoIDSchemaID != 0 { + return t.AutoIDSchemaID } return dbID } diff --git a/pkg/ddl/db_rename_test.go b/pkg/ddl/db_rename_test.go new file mode 100644 index 0000000000000..4484996796f10 --- /dev/null +++ b/pkg/ddl/db_rename_test.go @@ -0,0 +1,482 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl_test + +import ( + "fmt" + "testing" + gotime "time" + + "github.com/pingcap/tidb/pkg/config" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/store/mockstore" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +// See issue: https://github.com/pingcap/tidb/issues/29752 +// Ref https://dev.mysql.com/doc/refman/8.0/en/rename-table.html +func TestRenameTableWithLocked(t *testing.T) { + defer config.RestoreFunc()() + config.UpdateGlobal(func(conf *config.Config) { + conf.EnableTableLock = true + }) + + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database renamedb") + tk.MustExec("create database renamedb2") + tk.MustExec("use renamedb") + tk.MustExec("DROP TABLE IF EXISTS t1;") + tk.MustExec("CREATE TABLE t1 (a int);") + + tk.MustExec("LOCK TABLES t1 WRITE;") + tk.MustGetErrCode("drop database renamedb2;", errno.ErrLockOrActiveTransaction) + tk.MustExec("RENAME TABLE t1 TO t2;") + tk.MustQuery("select * from renamedb.t2").Check(testkit.Rows()) + tk.MustExec("UNLOCK TABLES") + tk.MustExec("RENAME TABLE t2 TO t1;") + tk.MustQuery("select * from renamedb.t1").Check(testkit.Rows()) + + tk.MustExec("LOCK TABLES t1 READ;") + tk.MustGetErrCode("RENAME TABLE t1 TO t2;", errno.ErrTableNotLockedForWrite) + tk.MustExec("UNLOCK TABLES") + + tk.MustExec("drop database renamedb") +} + +func TestRenameTable2(t *testing.T) { + isAlterTable := false + renameTableTest(t, "rename table %s to %s", isAlterTable) +} + +func TestAlterTableRenameTable(t *testing.T) { + isAlterTable := true + renameTableTest(t, "alter table %s rename to %s", isAlterTable) +} + +func renameTableTest(t *testing.T, sql string, isAlterTable bool) { + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustGetErrCode("rename table tb1 to tb2;", errno.ErrNoSuchTable) + // for different databases + tk.MustExec("create table t (c1 int, c2 int)") + tk.MustExec("insert t values (1, 1), (2, 2)") + ctx := tk.Session() + is := domain.GetDomain(ctx).InfoSchema() + oldTblInfo, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + oldTblID := oldTblInfo.Meta().ID + tk.MustExec("create database test1") + tk.MustExec("use test1") + tk.MustExec(fmt.Sprintf(sql, "test.t", "test1.t1")) + is = domain.GetDomain(ctx).InfoSchema() + newTblInfo, err := is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, oldTblID, newTblInfo.Meta().ID) + tk.MustQuery("select * from t1").Check(testkit.Rows("1 1", "2 2")) + tk.MustExec("use test") + + // Make sure t doesn't exist. + tk.MustExec("create table t (c1 int, c2 int)") + tk.MustExec("drop table t") + + // for the same database + tk.MustExec("use test1") + tk.MustExec(fmt.Sprintf(sql, "t1", "t2")) + is = domain.GetDomain(ctx).InfoSchema() + newTblInfo, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t2")) + require.NoError(t, err) + require.Equal(t, oldTblID, newTblInfo.Meta().ID) + tk.MustQuery("select * from t2").Check(testkit.Rows("1 1", "2 2")) + isExist := is.TableExists(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.False(t, isExist) + tk.MustQuery("show tables").Check(testkit.Rows("t2")) + + // for failure case + failSQL := fmt.Sprintf(sql, "test_not_exist.t", "test_not_exist.t") + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + failSQL = fmt.Sprintf(sql, "test.test_not_exist", "test.test_not_exist") + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + failSQL = fmt.Sprintf(sql, "test.t_not_exist", "test_not_exist.t") + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + failSQL = fmt.Sprintf(sql, "test1.t2", "test_not_exist.t") + tk.MustGetErrCode(failSQL, errno.ErrErrorOnRename) + + tk.MustExec("use test1") + tk.MustExec("create table if not exists t_exist (c1 int, c2 int)") + failSQL = fmt.Sprintf(sql, "test1.t2", "test1.t_exist") + tk.MustGetErrCode(failSQL, errno.ErrTableExists) + failSQL = fmt.Sprintf(sql, "test.t_not_exist", "test1.t_exist") + if isAlterTable { + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + } else { + tk.MustGetErrCode(failSQL, errno.ErrTableExists) + } + failSQL = fmt.Sprintf(sql, "test_not_exist.t", "test1.t_exist") + if isAlterTable { + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + } else { + tk.MustGetErrCode(failSQL, errno.ErrTableExists) + } + failSQL = fmt.Sprintf(sql, "test_not_exist.t", "test1.t_not_exist") + if isAlterTable { + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + } + + // for the same table name + tk.MustExec("use test1") + tk.MustExec("create table if not exists t (c1 int, c2 int)") + tk.MustExec("create table if not exists t1 (c1 int, c2 int)") + if isAlterTable { + tk.MustExec(fmt.Sprintf(sql, "test1.t", "t")) + tk.MustExec(fmt.Sprintf(sql, "test1.t1", "test1.T1")) + } else { + tk.MustGetErrCode(fmt.Sprintf(sql, "test1.t", "t"), errno.ErrTableExists) + tk.MustGetErrCode(fmt.Sprintf(sql, "test1.t1", "test1.T1"), errno.ErrTableExists) + } + + // Test rename table name too long. + tk.MustGetErrCode("rename table test1.t1 to test1.txxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx", errno.ErrTooLongIdent) + tk.MustGetErrCode("alter table test1.t1 rename to test1.txxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxxx", errno.ErrTooLongIdent) + + tk.MustExec("drop database test1") +} + +func TestRenameMultiTables(t *testing.T) { + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1(id int)") + tk.MustExec("create table t2(id int)") + sql := "rename table t1 to t3, t2 to t4" + _, err := tk.Exec(sql) + require.NoError(t, err) + + tk.MustExec("drop table t3, t4") + + tk.MustExec("create table t1 (c1 int, c2 int)") + tk.MustExec("create table t2 (c1 int, c2 int)") + tk.MustExec("insert t1 values (1, 1), (2, 2)") + tk.MustExec("insert t2 values (1, 1), (2, 2)") + ctx := tk.Session() + is := domain.GetDomain(ctx).InfoSchema() + oldTblInfo1, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + oldTblID1 := oldTblInfo1.Meta().ID + oldTblInfo2, err := is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + oldTblID2 := oldTblInfo2.Meta().ID + tk.MustExec("create database test1") + tk.MustExec("use test1") + tk.MustExec("rename table test.t1 to test1.t1, test.t2 to test1.t2") + is = domain.GetDomain(ctx).InfoSchema() + newTblInfo1, err := is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) + newTblInfo2, err := is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t2")) + require.NoError(t, err) + require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) + tk.MustQuery("select * from t1").Check(testkit.Rows("1 1", "2 2")) + tk.MustQuery("select * from t2").Check(testkit.Rows("1 1", "2 2")) + + // Make sure t1,t2 doesn't exist. + isExist := is.TableExists(model.NewCIStr("test"), model.NewCIStr("t1")) + require.False(t, isExist) + isExist = is.TableExists(model.NewCIStr("test"), model.NewCIStr("t2")) + require.False(t, isExist) + + // for the same database + tk.MustExec("use test1") + tk.MustExec("rename table test1.t1 to test1.t3, test1.t2 to test1.t4") + is = domain.GetDomain(ctx).InfoSchema() + newTblInfo1, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t3")) + require.NoError(t, err) + require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) + newTblInfo2, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t4")) + require.NoError(t, err) + require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) + tk.MustQuery("select * from t3").Check(testkit.Rows("1 1", "2 2")) + isExist = is.TableExists(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.False(t, isExist) + tk.MustQuery("select * from t4").Check(testkit.Rows("1 1", "2 2")) + isExist = is.TableExists(model.NewCIStr("test1"), model.NewCIStr("t2")) + require.False(t, isExist) + tk.MustQuery("show tables").Check(testkit.Rows("t3", "t4")) + + // for multi tables same database + tk.MustExec("create table t5 (c1 int, c2 int)") + tk.MustExec("insert t5 values (1, 1), (2, 2)") + is = domain.GetDomain(ctx).InfoSchema() + oldTblInfo3, err := is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t5")) + require.NoError(t, err) + oldTblID3 := oldTblInfo3.Meta().ID + tk.MustExec("rename table test1.t3 to test1.t1, test1.t4 to test1.t2, test1.t5 to test1.t3") + is = domain.GetDomain(ctx).InfoSchema() + newTblInfo1, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) + newTblInfo2, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t2")) + require.NoError(t, err) + require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) + newTblInfo3, err := is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t3")) + require.NoError(t, err) + require.Equal(t, oldTblID3, newTblInfo3.Meta().ID) + tk.MustQuery("show tables").Check(testkit.Rows("t1", "t2", "t3")) + + // for multi tables different databases + tk.MustExec("use test") + tk.MustExec("rename table test1.t1 to test.t2, test1.t2 to test.t3, test1.t3 to test.t4") + is = domain.GetDomain(ctx).InfoSchema() + newTblInfo1, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t2")) + require.NoError(t, err) + require.Equal(t, oldTblID1, newTblInfo1.Meta().ID) + newTblInfo2, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t3")) + require.NoError(t, err) + require.Equal(t, oldTblID2, newTblInfo2.Meta().ID) + newTblInfo3, err = is.TableByName(model.NewCIStr("test"), model.NewCIStr("t4")) + require.NoError(t, err) + require.Equal(t, oldTblID3, newTblInfo3.Meta().ID) + tk.MustQuery("show tables").Check(testkit.Rows("t2", "t3", "t4")) + + // for failure case + failSQL := "rename table test_not_exist.t to test_not_exist.t, test_not_exist.t to test_not_exist.t" + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + failSQL = "rename table test.test_not_exist to test.test_not_exist, test.test_not_exist to test.test_not_exist" + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + failSQL = "rename table test.t_not_exist to test_not_exist.t, test.t_not_exist to test_not_exist.t" + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + failSQL = "rename table test1.t2 to test_not_exist.t, test1.t2 to test_not_exist.t" + tk.MustGetErrCode(failSQL, errno.ErrNoSuchTable) + + tk.MustExec("drop database test1") + tk.MustExec("drop database test") +} + +func TestRenameMultiTablesIssue47064(t *testing.T) { + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t1(a int)") + tk.MustExec("create table t2(a int)") + tk.MustExec("create database test1") + tk.MustExec("rename table test.t1 to test1.t1, test.t2 to test1.t2") + tk.MustQuery("select column_name from information_schema.columns where table_name = 't1'").Check(testkit.Rows("a")) +} + +func TestRenameConcurrentAutoID(t *testing.T) { + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk1 := testkit.NewTestKit(t, store) + tk1.MustExec("use test") + // Use first client session, tidb1 + tk1.MustExec(`create schema if not exists test1`) + tk1.MustExec(`create schema if not exists test2`) + tk1.MustExec(`drop table if exists test1.t1, test2.t2`) + tk1.MustExec(`CREATE TABLE test1.t1 (a int auto_increment primary key nonclustered, b varchar(255), key (b)) auto_id_cache 5`) + tk1.MustExec(`begin`) + tk1.MustExec(`insert into test1.t1 values (null, "t1 first null")`) + tk1.MustQuery(`select _tidb_rowid, a, b from test1.t1`).Sort().Check(testkit.Rows("2 1 t1 first null")) + + ctx := tk1.Session() + is := domain.GetDomain(ctx).InfoSchema() + tblInfo, err := is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, int64(0), tblInfo.Meta().AutoIDSchemaID) + origAllocs := tblInfo.Allocators(nil) + require.Equal(t, int64(5), origAllocs.Allocs[0].End()) + + // Switch to a new client (tidb2) + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`use test`) + alterChan := make(chan error) + go func() { + // will wait for tidb1 + alterChan <- tk2.ExecToErr(`rename table test1.t1 to test2.t2`) + }() + tk3 := testkit.NewTestKit(t, store) + waitFor := func(tableName, s string, pos int) { + for { + select { + case alterErr := <-alterChan: + require.Fail(t, "Alter completed unexpectedly", "With error %v", alterErr) + default: + // Alter still running + } + res := tk3.MustQuery(`admin show ddl jobs where table_name = '` + tableName + `' and job_type = 'rename table'`).Rows() + if len(res) == 1 && res[0][pos] == s { + logutil.BgLogger().Info("Got state", zap.String("State", s)) + break + } + gotime.Sleep(50 * gotime.Millisecond) + } + // Sleep 50ms to wait load InforSchema finish, issue #46815. + gotime.Sleep(50 * gotime.Millisecond) + } + + // Switch to new client (tidb3) + waitFor("t1", "public", 4) + tk3.MustExec(`use test`) + tk3.MustExec(`begin`) + tk3.MustExec(`insert into test2.t2 values (null, "t2 first null")`) + tk3.MustQuery(`select _tidb_rowid, a, b from test2.t2`).Sort().Check(testkit.Rows("4 3 t2 first null")) + + // Switch back to tidb1 + // instead of generating 30k inserts with null + tk1.MustExec(`insert into test1.t1 values (null, "t1 second null")`) + // Bug was that this gave: + // ERROR 1146 (42S02): table doesn't exist + // Due to AutoID does no-longer exists. + tk1.MustExec(`insert into test1.t1 values (null, "t1 third null")`) + tk1.MustExec(`commit`) + tk3.MustExec(`insert into test2.t2 values (null, "t2 second null")`) + tk3.MustExec(`insert into test2.t2 values (null, "t2 third null")`) + tk3.MustExec(`commit`) + require.NoError(t, <-alterChan) + tk1.MustQuery(`select _tidb_rowid, a, b from test2.t2`).Sort().Check(testkit.Rows(""+ + "10 9 t2 second null", + "12 11 t2 third null", + "2 1 t1 first null", + "4 3 t2 first null", + "6 5 t1 second null", + "8 7 t1 third null")) + + // Unit test part for checking what happens when you rename back to the old schema (it should reset the 'AutoIDSchemaID' variable) + // and if you rename multiple time (so it does not lose the autoID). + ctx = tk1.Session() + is = domain.GetDomain(ctx).InfoSchema() + tblInfo, err = is.TableByName(model.NewCIStr("test2"), model.NewCIStr("t2")) + require.NoError(t, err) + originalSchemaID := tblInfo.Meta().AutoIDSchemaID + require.NotEqual(t, int64(0), originalSchemaID) + origAllocs = tblInfo.Allocators(nil) + require.Equal(t, int64(15), origAllocs.Allocs[0].End()) + + // Plan: + // - Rename to new table name in same Schema + // - Rename to new table name in new Schema + // - Rename to new table name in original Schema + // - Rename to new table name in new Schema + // - Drop original schema (verify that it does not clean up AutoIDs or hides them!) + // - Recreate original schema (by name) (Original Schema ID will not be used by anything else, ever!) + // - Rename to new table name in original Schema (should keep its AutoIDSchemaID) + + tk1.MustExec(`use test`) + tk1.MustExec(`rename table test2.t2 to test2.t1`) + tk1.MustExec(`insert into test2.t1 values (null, "Now t1 again")`) + tk1.MustQuery(`select _tidb_rowid, a, b from test2.t1`).Sort().Check(testkit.Rows(""+ + "10 9 t2 second null", + "12 11 t2 third null", + "14 13 Now t1 again", + "2 1 t1 first null", + "4 3 t2 first null", + "6 5 t1 second null", + "8 7 t1 third null")) + + ctx = tk1.Session() + is = domain.GetDomain(ctx).InfoSchema() + tblInfo, err = is.TableByName(model.NewCIStr("test2"), model.NewCIStr("t1")) + require.NoError(t, err) + require.Equal(t, originalSchemaID, tblInfo.Meta().AutoIDSchemaID) + origAllocs = tblInfo.Allocators(nil) + require.Equal(t, int64(15), origAllocs.Allocs[0].End()) + + tk1.MustExec(`insert into test2.t1 values (15, "Now t1, Explicit 15")`) + tk1.MustExec(`insert into test2.t1 values (null, "Is it 17?")`) + tk1.MustQuery(`select _tidb_rowid, a, b from test2.t1`).Sort().Check(testkit.Rows(""+ + "10 9 t2 second null", + "12 11 t2 third null", + "14 13 Now t1 again", + "16 15 Now t1, Explicit 15", + "18 17 Is it 17?", + "2 1 t1 first null", + "4 3 t2 first null", + "6 5 t1 second null", + "8 7 t1 third null")) + + tk1.MustExec(`rename table test2.t1 to test1.t1`) + + tk1.MustExec(`insert into test1.t1 values (null, "Is it 19?")`) + tk1.MustExec(`insert into test1.t1 values (22, "Now test1, Explicit 22")`) + tk1.MustExec(`insert into test1.t1 values (null, "Is it 24?")`) + tk1.MustQuery(`select _tidb_rowid, a, b from test1.t1`).Sort().Check(testkit.Rows(""+ + "10 9 t2 second null", + "12 11 t2 third null", + "14 13 Now t1 again", + "16 15 Now t1, Explicit 15", + "18 17 Is it 17?", + "2 1 t1 first null", + "20 19 Is it 19?", + "23 22 Now test1, Explicit 22", + "25 24 Is it 24?", + "4 3 t2 first null", + "6 5 t1 second null", + "8 7 t1 third null")) + + ctx = tk1.Session() + is = domain.GetDomain(ctx).InfoSchema() + tblInfo, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.NoError(t, err) + // Should be cleared when moved back to the original SchemaID + require.Equal(t, int64(0), tblInfo.Meta().AutoIDSchemaID) + + tk1.MustExec(`rename table test1.t1 to test2.t2`) + tk1.MustExec(`drop schema test1`) + tk1.MustExec(`insert into test2.t2 values (30, "Now test2 again, Explicit 30")`) + tk1.MustExec(`insert into test2.t2 values (null, "Is it 32?")`) + tk1.MustExec(`rename table test2.t2 to test2.t1`) + tk1.MustExec(`insert into test2.t1 values (35, "Now t1 again, Explicit 35")`) + tk1.MustExec(`insert into test2.t1 values (null, "Is it 37?")`) + tk1.MustExec(`create schema test1`) + tk1.MustExec(`rename table test2.t1 to test1.t1`) + + ctx = tk1.Session() + is = domain.GetDomain(ctx).InfoSchema() + tblInfo, err = is.TableByName(model.NewCIStr("test1"), model.NewCIStr("t1")) + require.NoError(t, err) + require.NotEqual(t, int64(0), tblInfo.Meta().AutoIDSchemaID) + origAllocs = tblInfo.Allocators(nil) + require.Equal(t, int64(40), origAllocs.Allocs[0].End()) + + tk1.MustExec(`insert into test1.t1 values (null, "Is it 39?")`) + + tk1.MustQuery(`select _tidb_rowid, a, b from test1.t1`).Sort().Check(testkit.Rows(""+ + "10 9 t2 second null", + "12 11 t2 third null", + "14 13 Now t1 again", + "16 15 Now t1, Explicit 15", + "18 17 Is it 17?", + "2 1 t1 first null", + "20 19 Is it 19?", + "23 22 Now test1, Explicit 22", + "25 24 Is it 24?", + "31 30 Now test2 again, Explicit 30", + "33 32 Is it 32?", + "36 35 Now t1 again, Explicit 35", + "38 37 Is it 37?", + "4 3 t2 first null", + "40 39 Is it 39?", + "6 5 t1 second null", + "8 7 t1 third null")) +} diff --git a/pkg/ddl/schema_test.go b/pkg/ddl/schema_test.go new file mode 100644 index 0000000000000..9b9929438f6be --- /dev/null +++ b/pkg/ddl/schema_test.go @@ -0,0 +1,451 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl_test + +import ( + "context" + "fmt" + "strings" + "testing" + "time" + + "github.com/ngaut/pools" + "github.com/pingcap/tidb/pkg/ddl" + "github.com/pingcap/tidb/pkg/infoschema" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/parser/terror" + "github.com/pingcap/tidb/pkg/sessionctx" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/logutil" + "github.com/stretchr/testify/require" + "go.uber.org/zap" +) + +func testCreateTable(t *testing.T, ctx sessionctx.Context, d ddl.DDL, dbInfo *model.DBInfo, tblInfo *model.TableInfo) *model.Job { + job := &model.Job{ + SchemaID: dbInfo.ID, + TableID: tblInfo.ID, + Type: model.ActionCreateTable, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{tblInfo}, + } + ctx.SetValue(sessionctx.QueryString, "skip") + err := d.DoDDLJob(ctx, job) + require.NoError(t, err) + + v := getSchemaVer(t, ctx) + tblInfo.State = model.StatePublic + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, tbl: tblInfo}) + tblInfo.State = model.StateNone + return job +} + +func testCheckTableState(t *testing.T, store kv.Storage, dbInfo *model.DBInfo, tblInfo *model.TableInfo, state model.SchemaState) { + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + require.NoError(t, kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + info, err := m.GetTable(dbInfo.ID, tblInfo.ID) + require.NoError(t, err) + + if state == model.StateNone { + require.NoError(t, err) + return nil + } + + require.Equal(t, info.Name, tblInfo.Name) + require.Equal(t, info.State, state) + return nil + })) +} + +// testTableInfo creates a test table with num int columns and with no index. +func testTableInfo(store kv.Storage, name string, num int) (*model.TableInfo, error) { + tblInfo := &model.TableInfo{ + Name: model.NewCIStr(name), + } + genIDs, err := genGlobalIDs(store, 1) + + if err != nil { + return nil, err + } + tblInfo.ID = genIDs[0] + + cols := make([]*model.ColumnInfo, num) + for i := range cols { + col := &model.ColumnInfo{ + Name: model.NewCIStr(fmt.Sprintf("c%d", i+1)), + Offset: i, + DefaultValue: i + 1, + State: model.StatePublic, + } + + col.FieldType = *types.NewFieldType(mysql.TypeLong) + tblInfo.MaxColumnID++ + col.ID = tblInfo.MaxColumnID + cols[i] = col + } + tblInfo.Columns = cols + tblInfo.Charset = "utf8" + tblInfo.Collate = "utf8_bin" + return tblInfo, nil +} + +func genGlobalIDs(store kv.Storage, count int) ([]int64, error) { + var ret []int64 + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + err := kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + var err error + ret, err = m.GenGlobalIDs(count) + return err + }) + return ret, err +} + +func testSchemaInfo(store kv.Storage, name string) (*model.DBInfo, error) { + dbInfo := &model.DBInfo{ + Name: model.NewCIStr(name), + } + + genIDs, err := genGlobalIDs(store, 1) + if err != nil { + return nil, err + } + dbInfo.ID = genIDs[0] + return dbInfo, nil +} + +func testCreateSchema(t *testing.T, ctx sessionctx.Context, d ddl.DDL, dbInfo *model.DBInfo) *model.Job { + job := &model.Job{ + SchemaID: dbInfo.ID, + Type: model.ActionCreateSchema, + BinlogInfo: &model.HistoryInfo{}, + Args: []interface{}{dbInfo}, + } + ctx.SetValue(sessionctx.QueryString, "skip") + require.NoError(t, d.DoDDLJob(ctx, job)) + + v := getSchemaVer(t, ctx) + dbInfo.State = model.StatePublic + checkHistoryJobArgs(t, ctx, job.ID, &historyJobArgs{ver: v, db: dbInfo}) + dbInfo.State = model.StateNone + return job +} + +func buildDropSchemaJob(dbInfo *model.DBInfo) *model.Job { + return &model.Job{ + SchemaID: dbInfo.ID, + Type: model.ActionDropSchema, + BinlogInfo: &model.HistoryInfo{}, + } +} + +func testDropSchema(t *testing.T, ctx sessionctx.Context, d ddl.DDL, dbInfo *model.DBInfo) (*model.Job, int64) { + job := buildDropSchemaJob(dbInfo) + ctx.SetValue(sessionctx.QueryString, "skip") + err := d.DoDDLJob(ctx, job) + require.NoError(t, err) + ver := getSchemaVer(t, ctx) + return job, ver +} + +func isDDLJobDone(test *testing.T, t *meta.Meta, store kv.Storage) bool { + tk := testkit.NewTestKit(test, store) + rows := tk.MustQuery("select * from mysql.tidb_ddl_job").Rows() + + if len(rows) == 0 { + return true + } + time.Sleep(testLease) + return false +} + +func testCheckSchemaState(test *testing.T, store kv.Storage, dbInfo *model.DBInfo, state model.SchemaState) { + isDropped := true + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + for { + err := kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + t := meta.NewMeta(txn) + info, err := t.GetDatabase(dbInfo.ID) + require.NoError(test, err) + + if state == model.StateNone { + isDropped = isDDLJobDone(test, t, store) + if !isDropped { + return nil + } + require.Nil(test, info) + return nil + } + + require.Equal(test, info.Name, dbInfo.Name) + require.Equal(test, info.State, state) + return nil + }) + require.NoError(test, err) + + if isDropped { + break + } + } +} + +func TestSchema(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease) + + dbInfo, err := testSchemaInfo(store, "test_schema") + require.NoError(t, err) + + // create a database. + tk := testkit.NewTestKit(t, store) + d := domain.DDL() + job := testCreateSchema(t, tk.Session(), d, dbInfo) + testCheckSchemaState(t, store, dbInfo, model.StatePublic) + testCheckJobDone(t, store, job.ID, true) + + /*** to drop the schema with two tables. ***/ + // create table t with 100 records. + tblInfo1, err := testTableInfo(store, "t", 3) + require.NoError(t, err) + tJob1 := testCreateTable(t, tk.Session(), d, dbInfo, tblInfo1) + testCheckTableState(t, store, dbInfo, tblInfo1, model.StatePublic) + testCheckJobDone(t, store, tJob1.ID, true) + tbl1 := testGetTable(t, domain, tblInfo1.ID) + err = sessiontxn.NewTxn(context.Background(), tk.Session()) + require.NoError(t, err) + for i := 1; i <= 100; i++ { + _, err := tbl1.AddRecord(tk.Session(), types.MakeDatums(i, i, i)) + require.NoError(t, err) + } + // create table t1 with 1034 records. + tblInfo2, err := testTableInfo(store, "t1", 3) + require.NoError(t, err) + tk2 := testkit.NewTestKit(t, store) + tJob2 := testCreateTable(t, tk2.Session(), d, dbInfo, tblInfo2) + testCheckTableState(t, store, dbInfo, tblInfo2, model.StatePublic) + testCheckJobDone(t, store, tJob2.ID, true) + tbl2 := testGetTable(t, domain, tblInfo2.ID) + err = sessiontxn.NewTxn(context.Background(), tk2.Session()) + require.NoError(t, err) + for i := 1; i <= 1034; i++ { + _, err := tbl2.AddRecord(tk2.Session(), types.MakeDatums(i, i, i)) + require.NoError(t, err) + } + tk3 := testkit.NewTestKit(t, store) + job, v := testDropSchema(t, tk3.Session(), d, dbInfo) + testCheckSchemaState(t, store, dbInfo, model.StateNone) + ids := make(map[int64]struct{}) + ids[tblInfo1.ID] = struct{}{} + ids[tblInfo2.ID] = struct{}{} + checkHistoryJobArgs(t, tk3.Session(), job.ID, &historyJobArgs{ver: v, db: dbInfo, tblIDs: ids}) + + // Drop a non-existent database. + job = &model.Job{ + SchemaID: dbInfo.ID, + Type: model.ActionDropSchema, + BinlogInfo: &model.HistoryInfo{}, + } + ctx := testkit.NewTestKit(t, store).Session() + ctx.SetValue(sessionctx.QueryString, "skip") + err = d.DoDDLJob(ctx, job) + require.True(t, terror.ErrorEqual(err, infoschema.ErrDatabaseDropExists), "err %v", err) + + // Drop a database without a table. + dbInfo1, err := testSchemaInfo(store, "test1") + require.NoError(t, err) + job = testCreateSchema(t, ctx, d, dbInfo1) + testCheckSchemaState(t, store, dbInfo1, model.StatePublic) + testCheckJobDone(t, store, job.ID, true) + job, _ = testDropSchema(t, ctx, d, dbInfo1) + testCheckSchemaState(t, store, dbInfo1, model.StateNone) + testCheckJobDone(t, store, job.ID, false) +} + +func TestSchemaWaitJob(t *testing.T) { + store, domain := testkit.CreateMockStoreAndDomainWithSchemaLease(t, testLease) + + require.True(t, domain.DDL().OwnerManager().IsOwner()) + + d2 := ddl.NewDDL(context.Background(), + ddl.WithEtcdClient(domain.EtcdClient()), + ddl.WithStore(store), + ddl.WithInfoCache(domain.InfoCache()), + ddl.WithLease(testLease), + ) + err := d2.Start(pools.NewResourcePool(func() (pools.Resource, error) { + session := testkit.NewTestKit(t, store).Session() + session.GetSessionVars().CommonGlobalLoaded = true + return session, nil + }, 20, 20, 5)) + require.NoError(t, err) + defer func() { + err := d2.Stop() + require.NoError(t, err) + }() + + // d2 must not be owner. + d2.OwnerManager().RetireOwner() + // wait one-second makes d2 stop pick up jobs. + time.Sleep(1 * time.Second) + + dbInfo, err := testSchemaInfo(store, "test_schema") + require.NoError(t, err) + se := testkit.NewTestKit(t, store).Session() + testCreateSchema(t, se, d2, dbInfo) + testCheckSchemaState(t, store, dbInfo, model.StatePublic) + + // d2 must not be owner. + require.False(t, d2.OwnerManager().IsOwner()) + + genIDs, err := genGlobalIDs(store, 1) + require.NoError(t, err) + schemaID := genIDs[0] + doDDLJobErr(t, schemaID, 0, model.ActionCreateSchema, []interface{}{dbInfo}, testkit.NewTestKit(t, store).Session(), d2, store) +} + +func doDDLJobErr(t *testing.T, schemaID, tableID int64, tp model.ActionType, args []interface{}, ctx sessionctx.Context, d ddl.DDL, store kv.Storage) *model.Job { + job := &model.Job{ + SchemaID: schemaID, + TableID: tableID, + Type: tp, + Args: args, + BinlogInfo: &model.HistoryInfo{}, + } + // TODO: check error detail + ctx.SetValue(sessionctx.QueryString, "skip") + require.Error(t, d.DoDDLJob(ctx, job)) + testCheckJobCancelled(t, store, job, nil) + + return job +} + +func testCheckJobCancelled(t *testing.T, store kv.Storage, job *model.Job, state *model.SchemaState) { + se := testkit.NewTestKit(t, store).Session() + historyJob, err := ddl.GetHistoryJobByID(se, job.ID) + require.NoError(t, err) + require.True(t, historyJob.IsCancelled() || historyJob.IsRollbackDone(), "history job %s", historyJob) + if state != nil { + require.Equal(t, historyJob.SchemaState, *state) + } +} + +func TestRenameTableAutoIDs(t *testing.T) { + store := testkit.CreateMockStore(t) + tk1 := testkit.NewTestKit(t, store) + + dbName := "RenameTableAutoIDs" + tk1.MustExec(`create schema ` + dbName) + tk1.MustExec(`create schema ` + dbName + "2") + tk1.MustExec(`use ` + dbName) + tk1.MustExec(`CREATE TABLE t (a int auto_increment primary key nonclustered, b varchar(255), key (b)) AUTO_ID_CACHE 100`) + tk1.MustExec(`insert into t values (11,11),(2,2),(null,12)`) + tk1.MustExec(`insert into t values (null,18)`) + tk1.MustQuery(`select _tidb_rowid, a, b from t`).Sort().Check(testkit.Rows("13 11 11", "14 2 2", "15 12 12", "17 16 18")) + + tk2 := testkit.NewTestKit(t, store) + tk2.MustExec(`use ` + dbName) + tk3 := testkit.NewTestKit(t, store) + tk3.MustExec(`use ` + dbName) + waitFor := func(col int, tableName, s string) { + for { + tk4 := testkit.NewTestKit(t, store) + tk4.MustExec(`use test`) + sql := `admin show ddl jobs where db_name like '` + strings.ToLower(dbName) + `%' and table_name like '` + tableName + `%' and job_type = 'rename table'` + res := tk4.MustQuery(sql).Rows() + if len(res) == 1 && res[0][col] == s { + break + } + logutil.BgLogger().Info("Could not find match", zap.String("tableName", tableName), zap.String("s", s), zap.Int("colNum", col)) + + for i := range res { + strs := make([]string, 0, len(res[i])) + for j := range res[i] { + strs = append(strs, res[i][j].(string)) + } + logutil.BgLogger().Info("ddl jobs", zap.Strings("jobs", strs)) + } + time.Sleep(10 * time.Millisecond) + } + } + alterChan := make(chan error) + tk2.MustExec(`set @@session.innodb_lock_wait_timeout = 0`) + tk2.MustExec(`BEGIN`) + tk2.MustExec(`insert into t values (null, 4)`) + go func() { + alterChan <- tk1.ExecToErr(`rename table t to ` + dbName + `2.t2`) + }() + waitFor(11, "t", "running") + waitFor(4, "t", "public") + tk3.MustExec(`BEGIN`) + tk3.MustExec(`insert into ` + dbName + `2.t2 values (50, 5)`) + + tk2.MustExec(`insert into t values (null, 6)`) + tk3.MustExec(`insert into ` + dbName + `2.t2 values (20, 5)`) + + // Done: Fix https://github.com/pingcap/tidb/issues/46904 + //tk2.MustContainErrMsg(`insert into t values (null, 6)`, "[tikv:1205]Lock wait timeout exceeded; try restarting transaction") + tk2.MustExec(`insert into t values (null, 6)`) + tk3.MustExec(`insert into ` + dbName + `2.t2 values (null, 7)`) + tk2.MustExec(`COMMIT`) + + waitFor(11, "t", "done") + tk2.MustExec(`BEGIN`) + tk2.MustExec(`insert into ` + dbName + `2.t2 values (null, 8)`) + + tk3.MustExec(`insert into ` + dbName + `2.t2 values (null, 9)`) + tk2.MustExec(`insert into ` + dbName + `2.t2 values (null, 10)`) + tk3.MustExec(`COMMIT`) + + waitFor(11, "t", "synced") + tk2.MustExec(`COMMIT`) + tk3.MustQuery(`select _tidb_rowid, a, b from ` + dbName + `2.t2`).Sort().Check(testkit.Rows(""+ + "13 11 11", + "14 2 2", + "15 12 12", + "17 16 18", + "19 18 4", + "51 50 5", + "53 52 6", + "54 20 5", + "56 55 6", + "58 57 7", + "60 59 8", + "62 61 9", + "64 63 10", + )) + + require.NoError(t, <-alterChan) + tk2.MustQuery(`select _tidb_rowid, a, b from ` + dbName + `2.t2`).Sort().Check(testkit.Rows(""+ + "13 11 11", + "14 2 2", + "15 12 12", + "17 16 18", + "19 18 4", + "51 50 5", + "53 52 6", + "54 20 5", + "56 55 6", + "58 57 7", + "60 59 8", + "62 61 9", + "64 63 10", + )) +} diff --git a/pkg/executor/test/autoidtest/autoid_test.go b/pkg/executor/test/autoidtest/autoid_test.go new file mode 100644 index 0000000000000..298989f60bb7b --- /dev/null +++ b/pkg/executor/test/autoidtest/autoid_test.go @@ -0,0 +1,569 @@ +// Copyright 2022 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package autoid_test + +import ( + "context" + "strconv" + "strings" + "testing" + + "github.com/pingcap/failpoint" + _ "github.com/pingcap/tidb/pkg/autoid_service" + ddltestutil "github.com/pingcap/tidb/pkg/ddl/testutil" + "github.com/pingcap/tidb/pkg/parser/mysql" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testutil" + "github.com/stretchr/testify/require" +) + +// Test filter different kind of allocators. +// In special ddl type, for example: +// 1: ActionRenameTable : it will abandon all the old allocators. +// 2: ActionRebaseAutoID : it will drop row-id-type allocator. +// 3: ActionModifyTableAutoIdCache : it will drop row-id-type allocator. +// 3: ActionRebaseAutoRandomBase : it will drop auto-rand-type allocator. +func TestFilterDifferentAllocators(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("drop table if exists t1") + + for _, str := range []string{"", " AUTO_ID_CACHE 1"} { + tk.MustExec("create table t(a bigint auto_random(5) key, b int auto_increment unique)" + str) + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("1")) + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles := testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Equal(t, int64(1), orderedHandles[0]) + tk.MustExec("delete from t") + + // Test rebase auto_increment. + tk.MustExec("alter table t auto_increment 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000000")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Equal(t, int64(2), orderedHandles[0]) + tk.MustExec("delete from t") + + // Test rebase auto_random. + tk.MustExec("alter table t auto_random_base 3000000") + tk.MustExec("insert into t values()") + tk.MustQuery("select b from t").Check(testkit.Rows("3000001")) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.Equal(t, int64(3000000), orderedHandles[0]) + tk.MustExec("delete from t") + + // Test rename table. + tk.MustExec("rename table t to t1") + tk.MustExec("insert into t1 values()") + res := tk.MustQuery("select b from t1") + strInt64, err := strconv.ParseInt(res.Rows()[0][0].(string), 10, 64) + require.NoError(t, err) + require.GreaterOrEqual(t, strInt64, int64(3000002)) + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "t1") + require.NoError(t, err) + require.Equal(t, 1, len(allHandles)) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + require.GreaterOrEqual(t, orderedHandles[0], int64(3000001)) + + tk.MustExec("drop table t1") + } +} + +func TestInsertWithAutoidSchema(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec(`use test`) + tk.MustExec(`create table t1(id int primary key auto_increment, n int);`) + tk.MustExec(`create table t2(id int unsigned primary key auto_increment, n int);`) + tk.MustExec(`create table t3(id tinyint primary key auto_increment, n int);`) + tk.MustExec(`create table t4(id int primary key, n float auto_increment, key I_n(n));`) + tk.MustExec(`create table t5(id int primary key, n float unsigned auto_increment, key I_n(n));`) + tk.MustExec(`create table t6(id int primary key, n double auto_increment, key I_n(n));`) + tk.MustExec(`create table t7(id int primary key, n double unsigned auto_increment, key I_n(n));`) + // test for inserting multiple values + tk.MustExec(`create table t8(id int primary key auto_increment, n int);`) + testInsertWithAutoidSchema(t, tk) + + // test for auto_id_cache = 1 + tk.MustExec(`drop table if exists t1, t2, t3, t4, t5, t6, t7, t8`) + tk.MustExec(`create table t1(id int primary key auto_increment, n int) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t2(id int unsigned primary key auto_increment, n int) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t3(id tinyint primary key auto_increment, n int) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t4(id int primary key, n float auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t5(id int primary key, n float unsigned auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t6(id int primary key, n double auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + tk.MustExec(`create table t7(id int primary key, n double unsigned auto_increment, key I_n(n)) AUTO_ID_CACHE 1;`) + // test for inserting multiple values + tk.MustExec(`create table t8(id int primary key auto_increment, n int);`) + testInsertWithAutoidSchema(t, tk) +} + +func testInsertWithAutoidSchema(t *testing.T, tk *testkit.TestKit) { + tests := []struct { + insert string + query string + result [][]interface{} + }{ + { + `insert into t1(id, n) values(1, 1)`, + `select * from t1 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t1(n) values(2)`, + `select * from t1 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t1(n) values(3)`, + `select * from t1 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t1(id, n) values(-1, 4)`, + `select * from t1 where id = -1`, + testkit.Rows(`-1 4`), + }, + { + `insert into t1(n) values(5)`, + `select * from t1 where id = 4`, + testkit.Rows(`4 5`), + }, + { + `insert into t1(id, n) values('5', 6)`, + `select * from t1 where id = 5`, + testkit.Rows(`5 6`), + }, + { + `insert into t1(n) values(7)`, + `select * from t1 where id = 6`, + testkit.Rows(`6 7`), + }, + { + `insert into t1(id, n) values(7.4, 8)`, + `select * from t1 where id = 7`, + testkit.Rows(`7 8`), + }, + { + `insert into t1(id, n) values(7.5, 9)`, + `select * from t1 where id = 8`, + testkit.Rows(`8 9`), + }, + { + `insert into t1(n) values(9)`, + `select * from t1 where id = 9`, + testkit.Rows(`9 9`), + }, + // test last insert id + { + `insert into t1 values(3000, -1), (null, -2)`, + `select * from t1 where id = 3000`, + testkit.Rows(`3000 -1`), + }, + { + `;`, + `select * from t1 where id = 3001`, + testkit.Rows(`3001 -2`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`3001`), + }, + { + `insert into t2(id, n) values(1, 1)`, + `select * from t2 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t2(n) values(2)`, + `select * from t2 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t2(n) values(3)`, + `select * from t2 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t3(id, n) values(1, 1)`, + `select * from t3 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t3(n) values(2)`, + `select * from t3 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t3(n) values(3)`, + `select * from t3 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t3(id, n) values(-1, 4)`, + `select * from t3 where id = -1`, + testkit.Rows(`-1 4`), + }, + { + `insert into t3(n) values(5)`, + `select * from t3 where id = 4`, + testkit.Rows(`4 5`), + }, + { + `insert into t4(id, n) values(1, 1)`, + `select * from t4 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t4(id) values(2)`, + `select * from t4 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t4(id, n) values(3, -1)`, + `select * from t4 where id = 3`, + testkit.Rows(`3 -1`), + }, + { + `insert into t4(id) values(4)`, + `select * from t4 where id = 4`, + testkit.Rows(`4 3`), + }, + { + `insert into t4(id, n) values(5, 5.5)`, + `select * from t4 where id = 5`, + testkit.Rows(`5 5.5`), + }, + { + `insert into t4(id) values(6)`, + `select * from t4 where id = 6`, + testkit.Rows(`6 7`), + }, + { + `insert into t4(id, n) values(7, '7.7')`, + `select * from t4 where id = 7`, + testkit.Rows(`7 7.7`), + }, + { + `insert into t4(id) values(8)`, + `select * from t4 where id = 8`, + testkit.Rows(`8 9`), + }, + { + `insert into t4(id, n) values(9, 10.4)`, + `select * from t4 where id = 9`, + testkit.Rows(`9 10.4`), + }, + { + `insert into t4(id) values(10)`, + `select * from t4 where id = 10`, + testkit.Rows(`10 11`), + }, + { + `insert into t5(id, n) values(1, 1)`, + `select * from t5 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t5(id) values(2)`, + `select * from t5 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t5(id) values(3)`, + `select * from t5 where id = 3`, + testkit.Rows(`3 3`), + }, + { + `insert into t6(id, n) values(1, 1)`, + `select * from t6 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t6(id) values(2)`, + `select * from t6 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t6(id, n) values(3, -1)`, + `select * from t6 where id = 3`, + testkit.Rows(`3 -1`), + }, + { + `insert into t6(id) values(4)`, + `select * from t6 where id = 4`, + testkit.Rows(`4 3`), + }, + { + `insert into t6(id, n) values(5, 5.5)`, + `select * from t6 where id = 5`, + testkit.Rows(`5 5.5`), + }, + { + `insert into t6(id) values(6)`, + `select * from t6 where id = 6`, + testkit.Rows(`6 7`), + }, + { + `insert into t6(id, n) values(7, '7.7')`, + `select * from t4 where id = 7`, + testkit.Rows(`7 7.7`), + }, + { + `insert into t6(id) values(8)`, + `select * from t4 where id = 8`, + testkit.Rows(`8 9`), + }, + { + `insert into t6(id, n) values(9, 10.4)`, + `select * from t6 where id = 9`, + testkit.Rows(`9 10.4`), + }, + { + `insert into t6(id) values(10)`, + `select * from t6 where id = 10`, + testkit.Rows(`10 11`), + }, + { + `insert into t7(id, n) values(1, 1)`, + `select * from t7 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `insert into t7(id) values(2)`, + `select * from t7 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `insert into t7(id) values(3)`, + `select * from t7 where id = 3`, + testkit.Rows(`3 3`), + }, + + // the following is test for insert multiple values. + { + `insert into t8(n) values(1),(2)`, + `select * from t8 where id = 1`, + testkit.Rows(`1 1`), + }, + { + `;`, + `select * from t8 where id = 2`, + testkit.Rows(`2 2`), + }, + { + `;`, + `select last_insert_id();`, + testkit.Rows(`1`), + }, + // test user rebase and auto alloc mixture. + { + `insert into t8 values(null, 3),(-1, -1),(null,4),(null, 5)`, + `select * from t8 where id = 3`, + testkit.Rows(`3 3`), + }, + // -1 won't rebase allocator here cause -1 < base. + { + `;`, + `select * from t8 where id = -1`, + testkit.Rows(`-1 -1`), + }, + { + `;`, + `select * from t8 where id = 4`, + testkit.Rows(`4 4`), + }, + { + `;`, + `select * from t8 where id = 5`, + testkit.Rows(`5 5`), + }, + { + `;`, + `select last_insert_id();`, + testkit.Rows(`3`), + }, + { + `insert into t8 values(null, 6),(10, 7),(null, 8)`, + `select * from t8 where id = 6`, + testkit.Rows(`6 6`), + }, + // 10 will rebase allocator here. + { + `;`, + `select * from t8 where id = 10`, + testkit.Rows(`10 7`), + }, + { + `;`, + `select * from t8 where id = 11`, + testkit.Rows(`11 8`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`6`), + }, + // fix bug for last_insert_id should be first allocated id in insert rows (skip the rebase id). + { + `insert into t8 values(100, 9),(null,10),(null,11)`, + `select * from t8 where id = 100`, + testkit.Rows(`100 9`), + }, + { + `;`, + `select * from t8 where id = 101`, + testkit.Rows(`101 10`), + }, + { + `;`, + `select * from t8 where id = 102`, + testkit.Rows(`102 11`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`101`), + }, + // test with sql_mode: NO_AUTO_VALUE_ON_ZERO. + { + `;`, + `select @@sql_mode`, + testkit.Rows(`ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`), + }, + { + `;`, + "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`", + nil, + }, + { + `insert into t8 values (0, 12), (null, 13)`, + `select * from t8 where id = 0`, + testkit.Rows(`0 12`), + }, + { + `;`, + `select * from t8 where id = 103`, + testkit.Rows(`103 13`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`103`), + }, + // test without sql_mode: NO_AUTO_VALUE_ON_ZERO. + { + `;`, + "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`", + nil, + }, + // value 0 will be substitute by autoid. + { + `insert into t8 values (0, 14), (null, 15)`, + `select * from t8 where id = 104`, + testkit.Rows(`104 14`), + }, + { + `;`, + `select * from t8 where id = 105`, + testkit.Rows(`105 15`), + }, + { + `;`, + `select last_insert_id()`, + testkit.Rows(`104`), + }, + // last test : auto increment allocation can find in retryInfo. + { + `retry : insert into t8 values (null, 16), (null, 17)`, + `select * from t8 where id = 1000`, + testkit.Rows(`1000 16`), + }, + { + `;`, + `select * from t8 where id = 1001`, + testkit.Rows(`1001 17`), + }, + { + `;`, + `select last_insert_id()`, + // this insert doesn't has the last_insert_id, should be same as the last insert case. + testkit.Rows(`104`), + }, + } + + for _, tt := range tests { + if strings.HasPrefix(tt.insert, "retry : ") { + // it's the last retry insert case, change the sessionVars. + retryInfo := &variable.RetryInfo{Retrying: true} + retryInfo.AddAutoIncrementID(1000) + retryInfo.AddAutoIncrementID(1001) + tk.Session().GetSessionVars().RetryInfo = retryInfo + tk.MustExec(tt.insert[8:]) + tk.Session().GetSessionVars().RetryInfo = &variable.RetryInfo{} + } else { + tk.MustExec(tt.insert) + } + if tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION,NO_AUTO_VALUE_ON_ZERO`" || + tt.query == "set session sql_mode = `ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION`" { + tk.MustExec(tt.query) + } else { + tk.MustQuery(tt.query).Check(tt.result) + } + } +} + +func TestMockAutoIDServiceError(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("USE test;") + tk.MustExec("create table t_mock_err (id int key auto_increment) auto_id_cache 1") + + failpoint.Enable("github.com/pingcap/tidb/pkg/autoid_service/mockErr", `return(true)`) + defer failpoint.Disable("github.com/pingcap/tidb/pkg/autoid_service/mockErr") + // Cover a bug that the autoid client retry non-retryable errors forever cause dead loop. + tk.MustExecToErr("insert into t_mock_err values (),()") // mock error, instead of dead loop +} + +func TestIssue39528(t *testing.T) { + // When AUTO_ID_CACHE is 1, it should not affect row id setting when autoid and rowid are separated. + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test;") + tk.MustExec("create table issue39528 (id int unsigned key nonclustered auto_increment) shard_row_id_bits=4 auto_id_cache 1;") + tk.MustExec("insert into issue39528 values ()") + tk.MustExec("insert into issue39528 values ()") + + ctx := context.Background() + var codeRun bool + ctx = context.WithValue(ctx, "testIssue39528", &codeRun) + _, err := tk.ExecWithContext(ctx, "insert into issue39528 values ()") + require.NoError(t, err) + // Make sure the code does not visit tikv on allocate path. + require.False(t, codeRun) +} diff --git a/pkg/executor/test/ddl/ddl_test.go b/pkg/executor/test/ddl/ddl_test.go new file mode 100644 index 0000000000000..8646c4634ac97 --- /dev/null +++ b/pkg/executor/test/ddl/ddl_test.go @@ -0,0 +1,1021 @@ +// Copyright 2016 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ddl + +import ( + "context" + "fmt" + "math" + "strconv" + "testing" + "time" + + "github.com/pingcap/failpoint" + "github.com/pingcap/tidb/pkg/ddl/schematracker" + ddltestutil "github.com/pingcap/tidb/pkg/ddl/testutil" + ddlutil "github.com/pingcap/tidb/pkg/ddl/util" + "github.com/pingcap/tidb/pkg/domain" + "github.com/pingcap/tidb/pkg/errno" + "github.com/pingcap/tidb/pkg/kv" + "github.com/pingcap/tidb/pkg/meta" + "github.com/pingcap/tidb/pkg/meta/autoid" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/pingcap/tidb/pkg/parser/mysql" + plannercore "github.com/pingcap/tidb/pkg/planner/core" + "github.com/pingcap/tidb/pkg/sessionctx/variable" + "github.com/pingcap/tidb/pkg/sessionctx/variable/featuretag/disttask" + "github.com/pingcap/tidb/pkg/sessiontxn" + "github.com/pingcap/tidb/pkg/store/mockstore" + "github.com/pingcap/tidb/pkg/table" + "github.com/pingcap/tidb/pkg/table/tables" + "github.com/pingcap/tidb/pkg/testkit" + "github.com/pingcap/tidb/pkg/testkit/testutil" + "github.com/pingcap/tidb/pkg/types" + "github.com/pingcap/tidb/pkg/util/chunk" + "github.com/pingcap/tidb/pkg/util/dbterror" + "github.com/stretchr/testify/require" +) + +// TestInTxnExecDDLFail tests the following case: +// 1. Execute the SQL of "begin"; +// 2. A SQL that will fail to execute; +// 3. Execute DDL. +func TestInTxnExecDDLFail(t *testing.T) { + store := testkit.CreateMockStore(t) + setTxnTk := testkit.NewTestKit(t, store) + setTxnTk.MustExec("set global tidb_txn_mode=''") + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table t (i int key);") + tk.MustExec("insert into t values (1);") + tk.MustExec("begin;") + tk.MustExec("insert into t values (1);") + tk.MustGetErrMsg("truncate table t;", "[kv:1062]Duplicate entry '1' for key 't.PRIMARY'") + tk.MustQuery("select count(*) from t").Check(testkit.Rows("1")) +} + +func TestCreateTable(t *testing.T) { + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + // Test create an exist database + tk.MustExecToErr("CREATE database test") + + // Test create an exist table + tk.MustExec("CREATE TABLE create_test (id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") + tk.MustExecToErr("CREATE TABLE create_test (id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") + + // Test "if not exist" + tk.MustExec("CREATE TABLE if not exists test(id INT NOT NULL DEFAULT 1, name varchar(255), PRIMARY KEY(id));") + + // Testcase for https://github.com/pingcap/tidb/issues/312 + tk.MustExec(`create table issue312_1 (c float(24));`) + tk.MustExec(`create table issue312_2 (c float(25));`) + rs, err := tk.Exec(`desc issue312_1`) + require.NoError(t, err) + ctx := context.Background() + req := rs.NewChunk(nil) + it := chunk.NewIterator4Chunk(req) + for { + err1 := rs.Next(ctx, req) + require.NoError(t, err1) + if req.NumRows() == 0 { + break + } + for row := it.Begin(); row != it.End(); row = it.Next() { + require.Equal(t, "float", row.GetString(1)) + } + } + rs, err = tk.Exec(`desc issue312_2`) + require.NoError(t, err) + req = rs.NewChunk(nil) + it = chunk.NewIterator4Chunk(req) + for { + err1 := rs.Next(ctx, req) + require.NoError(t, err1) + if req.NumRows() == 0 { + break + } + for row := it.Begin(); row != it.End(); row = it.Next() { + require.Equal(t, "double", req.GetRow(0).GetString(1)) + } + } + require.NoError(t, rs.Close()) + + // test multiple collate specified in column when create. + tk.MustExec("drop table if exists test_multiple_column_collate;") + tk.MustExec("create table test_multiple_column_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) + require.NoError(t, err) + require.Equal(t, "utf8", tt.Cols()[0].GetCharset()) + require.Equal(t, "utf8_general_ci", tt.Cols()[0].GetCollate()) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) + + tk.MustExec("drop table if exists test_multiple_column_collate;") + tk.MustExec("create table test_multiple_column_collate (a char(1) charset utf8 collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("test_multiple_column_collate")) + require.NoError(t, err) + require.Equal(t, "utf8", tt.Cols()[0].GetCharset()) + require.Equal(t, "utf8_general_ci", tt.Cols()[0].GetCollate()) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) + + // test Err case for multiple collate specified in column when create. + tk.MustExec("drop table if exists test_err_multiple_collate;") + tk.MustGetErrMsg("create table test_err_multiple_collate (a char(1) charset utf8mb4 collate utf8_unicode_ci collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin", + dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8_unicode_ci", "utf8mb4").Error()) + + tk.MustExec("drop table if exists test_err_multiple_collate;") + tk.MustGetErrMsg("create table test_err_multiple_collate (a char(1) collate utf8_unicode_ci collate utf8mb4_general_ci) charset utf8mb4 collate utf8mb4_bin", + dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8mb4_general_ci", "utf8").Error()) + + // table option is auto-increment + tk.MustExec("drop table if exists create_auto_increment_test;") + tk.MustExec("create table create_auto_increment_test (id int not null auto_increment, name varchar(255), primary key(id)) auto_increment = 999;") + tk.MustExec("insert into create_auto_increment_test (name) values ('aa')") + tk.MustExec("insert into create_auto_increment_test (name) values ('bb')") + tk.MustExec("insert into create_auto_increment_test (name) values ('cc')") + r := tk.MustQuery("select * from create_auto_increment_test;") + r.Check(testkit.Rows("999 aa", "1000 bb", "1001 cc")) + tk.MustExec("drop table create_auto_increment_test") + tk.MustExec("create table create_auto_increment_test (id int not null auto_increment, name varchar(255), primary key(id)) auto_increment = 1999;") + tk.MustExec("insert into create_auto_increment_test (name) values ('aa')") + tk.MustExec("insert into create_auto_increment_test (name) values ('bb')") + tk.MustExec("insert into create_auto_increment_test (name) values ('cc')") + r = tk.MustQuery("select * from create_auto_increment_test;") + r.Check(testkit.Rows("1999 aa", "2000 bb", "2001 cc")) + tk.MustExec("drop table create_auto_increment_test") + tk.MustExec("create table create_auto_increment_test (id int not null auto_increment, name varchar(255), key(id)) auto_increment = 1000;") + tk.MustExec("insert into create_auto_increment_test (name) values ('aa')") + r = tk.MustQuery("select * from create_auto_increment_test;") + r.Check(testkit.Rows("1000 aa")) + + // Test for `drop table if exists`. + tk.MustExec("drop table if exists t_if_exists;") + tk.MustQuery("show warnings;").Check(testkit.Rows("Note 1051 Unknown table 'test.t_if_exists'")) + tk.MustExec("create table if not exists t1_if_exists(c int)") + tk.MustExec("drop table if exists t1_if_exists,t2_if_exists,t3_if_exists") + tk.MustQuery("show warnings").Check(testkit.RowsWithSep("|", "Note|1051|Unknown table 'test.t2_if_exists'", "Note|1051|Unknown table 'test.t3_if_exists'")) +} + +func TestCreateDropDatabase(t *testing.T) { + store, dom := testkit.CreateMockStoreAndDomain(t, mockstore.WithDDLChecker()) + + ddlChecker := dom.DDL().(*schematracker.Checker) + + tk := testkit.NewTestKit(t, store) + tk.MustExec("create database if not exists drop_test;") + tk.MustExec("drop database if exists drop_test;") + tk.MustExec("create database drop_test;") + tk.MustExec("use drop_test;") + tk.MustExec("drop database drop_test;") + tk.MustGetDBError("drop table t;", plannercore.ErrNoDB) + tk.MustGetDBError("select * from t;", plannercore.ErrNoDB) + + tk.MustExecToErr("drop database mysql") + + tk.MustExec("create database charset_test charset ascii;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET ascii */", + )) + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test charset binary;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET binary */", + )) + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test collate utf8_general_ci;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET utf8 COLLATE utf8_general_ci */", + )) + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test charset utf8 collate utf8_general_ci;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET utf8 COLLATE utf8_general_ci */", + )) + tk.MustGetErrMsg("create database charset_test charset utf8 collate utf8mb4_unicode_ci;", "[ddl:1253]COLLATION 'utf8mb4_unicode_ci' is not valid for CHARACTER SET 'utf8'") + + // ddl.SchemaTracker will not respect session charset + ddlChecker.Disable() + + tk.MustExec("SET SESSION character_set_server='ascii'") + tk.MustExec("SET SESSION collation_server='ascii_bin'") + + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET ascii */", + )) + + ddlChecker.Enable() + + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test collate utf8mb4_general_ci;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET utf8mb4 COLLATE utf8mb4_general_ci */", + )) + + tk.MustExec("drop database charset_test;") + tk.MustExec("create database charset_test charset utf8mb4;") + tk.MustQuery("show create database charset_test;").Check(testkit.RowsWithSep("|", + "charset_test|CREATE DATABASE `charset_test` /*!40100 DEFAULT CHARACTER SET utf8mb4 */", + )) +} + +func TestAlterTableAddColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table if not exists alter_test (c1 int)") + tk.MustExec("insert into alter_test values(1)") + tk.MustExec("alter table alter_test add column c2 timestamp default current_timestamp") + time.Sleep(1 * time.Millisecond) + now := time.Now().Add(-1 * time.Millisecond).Format(types.TimeFormat) + r, err := tk.Exec("select c2 from alter_test") + require.NoError(t, err) + req := r.NewChunk(nil) + err = r.Next(context.Background(), req) + require.NoError(t, err) + row := req.GetRow(0) + require.Equal(t, 1, row.Len()) + require.GreaterOrEqual(t, now, row.GetTime(0).String()) + require.Nil(t, r.Close()) + tk.MustExec("alter table alter_test add column c3 varchar(50) default 'CURRENT_TIMESTAMP'") + tk.MustQuery("select c3 from alter_test").Check(testkit.Rows("CURRENT_TIMESTAMP")) + tk.MustExec("create or replace view alter_view as select c1,c2 from alter_test") + err = tk.ExecToErr("alter table alter_view add column c4 varchar(50)") + require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) + tk.MustExec("drop view alter_view") + tk.MustExec("create sequence alter_seq") + err = tk.ExecToErr("alter table alter_seq add column c int") + require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) + tk.MustExec("alter table alter_test add column c4 date default current_date") + now = time.Now().Format(types.DateFormat) + r, err = tk.Exec("select c4 from alter_test") + require.NoError(t, err) + req = r.NewChunk(nil) + err = r.Next(context.Background(), req) + require.NoError(t, err) + row = req.GetRow(0) + require.Equal(t, 1, row.Len()) + require.GreaterOrEqual(t, now, row.GetTime(0).String()) + require.Nil(t, r.Close()) + tk.MustExec("drop sequence alter_seq") +} + +func TestAlterTableAddColumns(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table if not exists alter_test (c1 int)") + tk.MustExec("insert into alter_test values(1)") + tk.MustExec("alter table alter_test add column c2 timestamp default current_timestamp, add column c8 varchar(50) default 'CURRENT_TIMESTAMP'") + tk.MustExec("alter table alter_test add column (c7 timestamp default current_timestamp, c3 varchar(50) default 'CURRENT_TIMESTAMP')") + r, err := tk.Exec("select c2 from alter_test") + require.NoError(t, err) + req := r.NewChunk(nil) + err = r.Next(context.Background(), req) + require.NoError(t, err) + row := req.GetRow(0) + require.Equal(t, 1, row.Len()) + require.Nil(t, r.Close()) + tk.MustQuery("select c3 from alter_test").Check(testkit.Rows("CURRENT_TIMESTAMP")) + tk.MustExec("create or replace view alter_view as select c1,c2 from alter_test") + err = tk.ExecToErr("alter table alter_view add column (c4 varchar(50), c5 varchar(50))") + require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error(), err.Error()) + tk.MustExec("drop view alter_view") + tk.MustExec("create sequence alter_seq") + err = tk.ExecToErr("alter table alter_seq add column (c1 int, c2 varchar(10))") + require.Equal(t, dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error(), err.Error()) + tk.MustExec("drop sequence alter_seq") +} + +func TestAddNotNullColumnNoDefault(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("create table nn (c1 int)") + tk.MustExec("insert nn values (1), (2)") + tk.MustExec("alter table nn add column c2 int not null") + + tbl, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("nn")) + require.NoError(t, err) + col2 := tbl.Meta().Columns[1] + require.Nil(t, col2.DefaultValue) + require.Equal(t, "0", col2.OriginDefaultValue) + + tk.MustQuery("select * from nn").Check(testkit.Rows("1 0", "2 0")) + tk.MustExecToErr("insert nn (c1) values (3)") + tk.MustExec("set sql_mode=''") + tk.MustExec("insert nn (c1) values (3)") + tk.MustQuery("select * from nn").Check(testkit.Rows("1 0", "2 0", "3 0")) +} + +func TestAlterTableModifyColumn(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + tk.MustExec("drop table if exists mc") + tk.MustExec("create table mc(c1 int, c2 varchar(10), c3 bit)") + tk.MustExecToErr("alter table mc modify column c1 short") + tk.MustExec("alter table mc modify column c1 bigint") + + tk.MustExecToErr("alter table mc modify column c2 blob") + tk.MustExec("alter table mc modify column c2 varchar(8)") + tk.MustExec("alter table mc modify column c2 varchar(11)") + tk.MustExec("alter table mc modify column c2 text(13)") + tk.MustExec("alter table mc modify column c2 text") + tk.MustExec("alter table mc modify column c3 bit") + result := tk.MustQuery("show create table mc") + createSQL := result.Rows()[0][1] + expected := "CREATE TABLE `mc` (\n `c1` bigint(20) DEFAULT NULL,\n `c2` text DEFAULT NULL,\n `c3` bit(1) DEFAULT NULL\n) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin" + require.Equal(t, expected, createSQL) + tk.MustExec("create or replace view alter_view as select c1,c2 from mc") + tk.MustGetErrMsg("alter table alter_view modify column c2 text", + dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_view", "BASE TABLE").Error()) + tk.MustExec("drop view alter_view") + tk.MustExec("create sequence alter_seq") + tk.MustGetErrMsg("alter table alter_seq modify column c int", + dbterror.ErrWrongObject.GenWithStackByArgs("test", "alter_seq", "BASE TABLE").Error()) + tk.MustExec("drop sequence alter_seq") + + // test multiple collate modification in column. + tk.MustExec("drop table if exists modify_column_multiple_collate") + tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") + tk.MustExec("alter table modify_column_multiple_collate modify column a char(1) collate utf8mb4_bin;") + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) + require.NoError(t, err) + require.Equal(t, "utf8mb4", tt.Cols()[0].GetCharset()) + require.Equal(t, "utf8mb4_bin", tt.Cols()[0].GetCollate()) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) + + tk.MustExec("drop table if exists modify_column_multiple_collate;") + tk.MustExec("create table modify_column_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") + tk.MustExec("alter table modify_column_multiple_collate modify column a char(1) charset utf8mb4 collate utf8mb4_bin;") + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("modify_column_multiple_collate")) + require.NoError(t, err) + require.Equal(t, "utf8mb4", tt.Cols()[0].GetCharset()) + require.Equal(t, "utf8mb4_bin", tt.Cols()[0].GetCollate()) + require.Equal(t, "utf8mb4", tt.Meta().Charset) + require.Equal(t, "utf8mb4_bin", tt.Meta().Collate) + + // test Err case for multiple collate modification in column. + tk.MustExec("drop table if exists err_modify_multiple_collate;") + tk.MustExec("create table err_modify_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") + tk.MustGetErrMsg("alter table err_modify_multiple_collate modify column a char(1) charset utf8mb4 collate utf8_bin;", dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8_bin", "utf8mb4").Error()) + + tk.MustExec("drop table if exists err_modify_multiple_collate;") + tk.MustExec("create table err_modify_multiple_collate (a char(1) collate utf8_bin collate utf8_general_ci) charset utf8mb4 collate utf8mb4_bin") + tk.MustGetErrMsg("alter table err_modify_multiple_collate modify column a char(1) collate utf8_bin collate utf8mb4_bin;", dbterror.ErrCollationCharsetMismatch.GenWithStackByArgs("utf8mb4_bin", "utf8").Error()) +} + +func TestColumnCharsetAndCollate(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + dbName := "col_charset_collate" + tk.MustExec("create database " + dbName) + tk.MustExec("use " + dbName) + tests := []struct { + colType string + charset string + collates string + exptCharset string + exptCollate string + errMsg string + }{ + { + colType: "varchar(10)", + charset: "charset utf8", + collates: "collate utf8_bin", + exptCharset: "utf8", + exptCollate: "utf8_bin", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset utf8mb4", + collates: "", + exptCharset: "utf8mb4", + exptCollate: "utf8mb4_bin", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset utf16", + collates: "", + exptCharset: "", + exptCollate: "", + errMsg: "Unknown charset utf16", + }, + { + colType: "varchar(10)", + charset: "charset latin1", + collates: "", + exptCharset: "latin1", + exptCollate: "latin1_bin", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset binary", + collates: "", + exptCharset: "binary", + exptCollate: "binary", + errMsg: "", + }, + { + colType: "varchar(10)", + charset: "charset ascii", + collates: "", + exptCharset: "ascii", + exptCollate: "ascii_bin", + errMsg: "", + }, + } + sctx := tk.Session() + dm := domain.GetDomain(sctx) + for i, tt := range tests { + tblName := fmt.Sprintf("t%d", i) + sql := fmt.Sprintf("create table %s (a %s %s %s)", tblName, tt.colType, tt.charset, tt.collates) + if tt.errMsg == "" { + tk.MustExec(sql) + is := dm.InfoSchema() + require.NotNil(t, is) + + tb, err := is.TableByName(model.NewCIStr(dbName), model.NewCIStr(tblName)) + require.NoError(t, err) + require.Equalf(t, tt.exptCharset, tb.Meta().Columns[0].GetCharset(), sql) + require.Equalf(t, tt.exptCollate, tb.Meta().Columns[0].GetCollate(), sql) + } else { + err := tk.ExecToErr(sql) + require.Errorf(t, err, sql) + } + } + tk.MustExec("drop database " + dbName) +} + +func TestShardRowIDBits(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("use test") + tk.MustExec("create table t (a int) shard_row_id_bits = 15") + for i := 0; i < 100; i++ { + tk.MustExec("insert into t values (?)", i) + } + + dom := domain.GetDomain(tk.Session()) + tbl, err := dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t")) + require.NoError(t, err) + + assertCountAndShard := func(tt table.Table, expectCount int) { + var hasShardedID bool + var count int + require.NoError(t, sessiontxn.NewTxn(context.Background(), tk.Session())) + err = tables.IterRecords(tt, tk.Session(), nil, func(h kv.Handle, rec []types.Datum, cols []*table.Column) (more bool, err error) { + require.GreaterOrEqual(t, h.IntValue(), int64(0)) + first8bits := h.IntValue() >> 56 + if first8bits > 0 { + hasShardedID = true + } + count++ + return true, nil + }) + require.NoError(t, err) + require.Equal(t, expectCount, count) + require.True(t, hasShardedID) + } + + assertCountAndShard(tbl, 100) + + // After PR 10759, shard_row_id_bits is supported with tables with auto_increment column. + tk.MustExec("create table auto (id int not null auto_increment unique) shard_row_id_bits = 4") + tk.MustExec("alter table auto shard_row_id_bits = 5") + tk.MustExec("drop table auto") + tk.MustExec("create table auto (id int not null auto_increment unique) shard_row_id_bits = 0") + tk.MustExec("alter table auto shard_row_id_bits = 5") + tk.MustExec("drop table auto") + tk.MustExec("create table auto (id int not null auto_increment unique)") + tk.MustExec("alter table auto shard_row_id_bits = 5") + tk.MustExec("drop table auto") + tk.MustExec("create table auto (id int not null auto_increment unique) shard_row_id_bits = 4") + tk.MustExec("alter table auto shard_row_id_bits = 0") + tk.MustExec("drop table auto") + + errMsg := "[ddl:8200]Unsupported shard_row_id_bits for table with primary key as row id" + tk.MustGetErrMsg("create table auto (id varchar(255) primary key clustered, b int) shard_row_id_bits = 4;", errMsg) + tk.MustExec("create table auto (id varchar(255) primary key clustered, b int) shard_row_id_bits = 0;") + tk.MustGetErrMsg("alter table auto shard_row_id_bits = 5;", errMsg) + tk.MustExec("alter table auto shard_row_id_bits = 0;") + tk.MustExec("drop table if exists auto;") + + // After PR 10759, shard_row_id_bits is not supported with pk_is_handle tables. + tk.MustGetErrMsg("create table auto (id int not null auto_increment primary key, b int) shard_row_id_bits = 4", errMsg) + tk.MustExec("create table auto (id int not null auto_increment primary key, b int) shard_row_id_bits = 0") + tk.MustGetErrMsg("alter table auto shard_row_id_bits = 5", errMsg) + tk.MustExec("alter table auto shard_row_id_bits = 0") + + // Hack an existing table with shard_row_id_bits and primary key as handle + db, ok := dom.InfoSchema().SchemaByName(model.NewCIStr("test")) + require.True(t, ok) + tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto")) + tblInfo := tbl.Meta() + tblInfo.ShardRowIDBits = 5 + tblInfo.MaxShardRowIDBits = 5 + + ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnDDL) + err = kv.RunInNewTxn(ctx, store, false, func(ctx context.Context, txn kv.Transaction) error { + m := meta.NewMeta(txn) + _, err = m.GenSchemaVersion() + require.NoError(t, err) + require.Nil(t, m.UpdateTable(db.ID, tblInfo)) + return nil + }) + require.NoError(t, err) + err = dom.Reload() + require.NoError(t, err) + + tk.MustExec("insert auto(b) values (1), (3), (5)") + tk.MustQuery("select id from auto order by id").Check(testkit.Rows("1", "2", "3")) + + tk.MustExec("alter table auto shard_row_id_bits = 0") + tk.MustExec("drop table auto") + + // Test shard_row_id_bits with auto_increment column + tk.MustExec("create table auto (a int, b int auto_increment unique) shard_row_id_bits = 15") + for i := 0; i < 100; i++ { + tk.MustExec("insert into auto(a) values (?)", i) + } + tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto")) + assertCountAndShard(tbl, 100) + prevB, err := strconv.Atoi(tk.MustQuery("select b from auto where a=0").Rows()[0][0].(string)) + require.NoError(t, err) + for i := 1; i < 100; i++ { + b, err := strconv.Atoi(tk.MustQuery(fmt.Sprintf("select b from auto where a=%d", i)).Rows()[0][0].(string)) + require.NoError(t, err) + require.Greater(t, b, prevB) + prevB = b + } + + // Test overflow + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int) shard_row_id_bits = 15") + defer tk.MustExec("drop table if exists t1") + + tbl, err = dom.InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("t1")) + require.NoError(t, err) + maxID := 1<<(64-15-1) - 1 + alloc := tbl.Allocators(tk.Session()).Get(autoid.RowIDAllocType) + err = alloc.Rebase(context.Background(), int64(maxID)-1, false) + require.NoError(t, err) + tk.MustExec("insert into t1 values(1)") + + // continue inserting will fail. + tk.MustGetDBError("insert into t1 values(2)", autoid.ErrAutoincReadFailed) + tk.MustGetDBError("insert into t1 values(3)", autoid.ErrAutoincReadFailed) +} + +func TestAutoRandomBitsData(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + + tk.MustExec("create database if not exists test_auto_random_bits") + defer tk.MustExec("drop database if exists test_auto_random_bits") + tk.MustExec("use test_auto_random_bits") + tk.MustExec("drop table if exists t") + + extractAllHandles := func() []int64 { + allHds, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test_auto_random_bits", "t") + require.NoError(t, err) + return allHds + } + + tk.MustExec("set @@allow_auto_random_explicit_insert = true") + + tk.MustExec("create table t (a bigint primary key clustered auto_random(15), b int)") + for i := 0; i < 100; i++ { + tk.MustExec("insert into t(b) values (?)", i) + } + allHandles := extractAllHandles() + tk.MustExec("drop table t") + + // Test auto random id number. + require.Equal(t, 100, len(allHandles)) + // Test the handles are not all zero. + allZero := true + for _, h := range allHandles { + allZero = allZero && (h>>(64-16)) == 0 + } + require.False(t, allZero) + // Test non-shard-bits part of auto random id is monotonic increasing and continuous. + orderedHandles := testutil.MaskSortHandles(allHandles, 15, mysql.TypeLonglong) + size := int64(len(allHandles)) + for i := int64(1); i <= size; i++ { + require.Equal(t, orderedHandles[i-1], i) + } + + // Test explicit insert. + autoRandBitsUpperBound := 2<<47 - 1 + tk.MustExec("create table t (a bigint primary key clustered auto_random(15), b int)") + for i := -10; i < 10; i++ { + tk.MustExec(fmt.Sprintf("insert into t values(%d, %d)", i+autoRandBitsUpperBound, i)) + } + tk.MustGetErrMsg("insert into t (b) values (0)", autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) + tk.MustExec("drop table t") + + // Test overflow. + tk.MustExec("create table t (a bigint primary key auto_random(15), b int)") + // Here we cannot fill the all values for a `bigint` column, + // so firstly we rebase auto_rand to the position before overflow. + tk.MustExec(fmt.Sprintf("insert into t values (%d, %d)", autoRandBitsUpperBound, 1)) + tk.MustGetErrMsg("insert into t (b) values (0)", autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) + tk.MustExec("drop table t") + + tk.MustExec("create table t (a bigint primary key auto_random(15), b int)") + tk.MustExec("insert into t values (1, 2)") + tk.MustExec(fmt.Sprintf("update t set a = %d where a = 1", autoRandBitsUpperBound)) + tk.MustGetErrMsg("insert into t (b) values (0)", autoid.ErrAutoRandReadFailed.GenWithStackByArgs().Error()) + tk.MustExec("drop table t") + + // Test insert negative integers explicitly won't trigger rebase. + tk.MustExec("create table t (a bigint primary key auto_random(15), b int)") + for i := 1; i <= 100; i++ { + tk.MustExec("insert into t(b) values (?)", i) + tk.MustExec("insert into t(a, b) values (?, ?)", -i, i) + } + // orderedHandles should be [-100, -99, ..., -2, -1, 1, 2, ..., 99, 100] + orderedHandles = testutil.MaskSortHandles(extractAllHandles(), 15, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < 100; i++ { + require.Equal(t, i-100, orderedHandles[i]) + } + for i := int64(100); i < size; i++ { + require.Equal(t, i-99, orderedHandles[i]) + } + tk.MustExec("drop table t") + + // Test signed/unsigned types. + tk.MustExec("create table t (a bigint primary key auto_random(10), b int)") + for i := 0; i < 100; i++ { + tk.MustExec("insert into t (b) values(?)", i) + } + for _, h := range extractAllHandles() { + // Sign bit should be reserved. + require.True(t, h > 0) + } + tk.MustExec("drop table t") + + tk.MustExec("create table t (a bigint unsigned primary key auto_random(10), b int)") + for i := 0; i < 100; i++ { + tk.MustExec("insert into t (b) values(?)", i) + } + signBitUnused := true + for _, h := range extractAllHandles() { + signBitUnused = signBitUnused && (h > 0) + } + // Sign bit should be used for shard. + require.False(t, signBitUnused) + tk.MustExec("drop table t;") + + // Test rename table does not affect incremental part of auto_random ID. + tk.MustExec("create database test_auto_random_bits_rename;") + tk.MustExec("create table t (a bigint auto_random primary key);") + for i := 0; i < 10; i++ { + tk.MustExec("insert into t values ();") + } + tk.MustExec("alter table t rename to test_auto_random_bits_rename.t1;") + for i := 0; i < 10; i++ { + tk.MustExec("insert into test_auto_random_bits_rename.t1 values ();") + } + tk.MustExec("alter table test_auto_random_bits_rename.t1 rename to t;") + for i := 0; i < 10; i++ { + tk.MustExec("insert into t values ();") + } + uniqueHandles := make(map[int64]struct{}) + for _, h := range extractAllHandles() { + uniqueHandles[h&((1<<(63-5))-1)] = struct{}{} + } + require.Equal(t, 30, len(uniqueHandles)) + tk.MustExec("drop database test_auto_random_bits_rename;") + tk.MustExec("drop table t;") +} + +func TestAutoRandomTableOption(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + + // test table option is auto-random + tk.MustExec("drop table if exists auto_random_table_option") + tk.MustExec("create table auto_random_table_option (a bigint auto_random(5) key) auto_random_base = 1000") + tt, err := domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("auto_random_table_option")) + require.NoError(t, err) + require.Equal(t, int64(1000), tt.Meta().AutoRandID) + tk.MustExec("insert into auto_random_table_option values (),(),(),(),()") + allHandles, err := ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "auto_random_table_option") + require.NoError(t, err) + require.Equal(t, 5, len(allHandles)) + // Test non-shard-bits part of auto random id is monotonic increasing and continuous. + orderedHandles := testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size := int64(len(allHandles)) + for i := int64(0); i < size; i++ { + require.Equal(t, orderedHandles[i], i+1000) + } + + tk.MustExec("drop table if exists alter_table_auto_random_option") + tk.MustExec("create table alter_table_auto_random_option (a bigint primary key auto_random(4), b int)") + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + require.NoError(t, err) + require.Equal(t, int64(0), tt.Meta().AutoRandID) + tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "alter_table_auto_random_option") + require.NoError(t, err) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < size; i++ { + require.Equal(t, i+1, orderedHandles[i]) + } + tk.MustExec("delete from alter_table_auto_random_option") + + // alter table to change the auto_random option (it will dismiss the local allocator cache) + // To avoid the new base is in the range of local cache, which will leading the next + // value is not what we rebased, because the local cache is dropped, here we choose + // a quite big value to do this. + tk.MustExec("alter table alter_table_auto_random_option auto_random_base = 3000000") + tt, err = domain.GetDomain(tk.Session()).InfoSchema().TableByName(model.NewCIStr("test"), model.NewCIStr("alter_table_auto_random_option")) + require.NoError(t, err) + require.Equal(t, int64(3000000), tt.Meta().AutoRandID) + tk.MustExec("insert into alter_table_auto_random_option values(),(),(),(),()") + allHandles, err = ddltestutil.ExtractAllTableHandles(tk.Session(), "test", "alter_table_auto_random_option") + require.NoError(t, err) + orderedHandles = testutil.MaskSortHandles(allHandles, 5, mysql.TypeLonglong) + size = int64(len(allHandles)) + for i := int64(0); i < size; i++ { + require.Equal(t, i+3000000, orderedHandles[i]) + } + tk.MustExec("drop table alter_table_auto_random_option") + + // Alter auto_random_base on non auto_random table. + tk.MustExec("create table alter_auto_random_normal (a int)") + err = tk.ExecToErr("alter table alter_auto_random_normal auto_random_base = 100") + require.Error(t, err) + require.Contains(t, err.Error(), autoid.AutoRandomRebaseNotApplicable) +} + +func TestSetDDLReorgWorkerCnt(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(variable.DefTiDBDDLReorgWorkerCount), variable.GetDDLReorgWorkerCounter()) + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 1") + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(1), variable.GetDDLReorgWorkerCounter()) + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) + tk.MustGetDBError("set @@global.tidb_ddl_reorg_worker_cnt = invalid_val", variable.ErrWrongTypeForVar) + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(100), variable.GetDDLReorgWorkerCounter()) + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = -1") + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_worker_cnt value: '-1'")) + tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("1")) + + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") + res := tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt") + res.Check(testkit.Rows("100")) + + res = tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt") + res.Check(testkit.Rows("100")) + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 100") + res = tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt") + res.Check(testkit.Rows("100")) + + tk.MustExec("set @@global.tidb_ddl_reorg_worker_cnt = 257") + tk.MustQuery("SHOW WARNINGS").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_worker_cnt value: '257'")) + tk.MustQuery("select @@global.tidb_ddl_reorg_worker_cnt").Check(testkit.Rows("256")) +} + +func TestSetDDLReorgBatchSize(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + err := ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(variable.DefTiDBDDLReorgBatchSize), variable.GetDDLReorgBatchSize()) + + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 1") + tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '1'")) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, variable.MinDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) + tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_reorg_batch_size = %v", variable.MaxDDLReorgBatchSize+1)) + tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '%d'", variable.MaxDDLReorgBatchSize+1))) + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, variable.MaxDDLReorgBatchSize, variable.GetDDLReorgBatchSize()) + tk.MustGetDBError("set @@global.tidb_ddl_reorg_batch_size = invalid_val", variable.ErrWrongTypeForVar) + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 100") + err = ddlutil.LoadDDLReorgVars(context.Background(), tk.Session()) + require.NoError(t, err) + require.Equal(t, int32(100), variable.GetDDLReorgBatchSize()) + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = -1") + tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_reorg_batch_size value: '-1'")) + + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 100") + res := tk.MustQuery("select @@global.tidb_ddl_reorg_batch_size") + res.Check(testkit.Rows("100")) + + res = tk.MustQuery("select @@global.tidb_ddl_reorg_batch_size") + res.Check(testkit.Rows(fmt.Sprintf("%v", 100))) + tk.MustExec("set @@global.tidb_ddl_reorg_batch_size = 1000") + res = tk.MustQuery("select @@global.tidb_ddl_reorg_batch_size") + res.Check(testkit.Rows("1000")) +} + +func TestSetDDLErrorCountLimit(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + err := ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(variable.DefTiDBDDLErrorCountLimit), variable.GetDDLErrorCountLimit()) + + tk.MustExec("set @@global.tidb_ddl_error_count_limit = -1") + tk.MustQuery("show warnings;").Check(testkit.Rows("Warning 1292 Truncated incorrect tidb_ddl_error_count_limit value: '-1'")) + err = ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(0), variable.GetDDLErrorCountLimit()) + tk.MustExec(fmt.Sprintf("set @@global.tidb_ddl_error_count_limit = %v", uint64(math.MaxInt64)+1)) + tk.MustQuery("show warnings;").Check(testkit.Rows(fmt.Sprintf("Warning 1292 Truncated incorrect tidb_ddl_error_count_limit value: '%d'", uint64(math.MaxInt64)+1))) + err = ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(math.MaxInt64), variable.GetDDLErrorCountLimit()) + tk.MustGetDBError("set @@global.tidb_ddl_error_count_limit = invalid_val", variable.ErrWrongTypeForVar) + tk.MustExec("set @@global.tidb_ddl_error_count_limit = 100") + err = ddlutil.LoadDDLVars(tk.Session()) + require.NoError(t, err) + require.Equal(t, int64(100), variable.GetDDLErrorCountLimit()) + res := tk.MustQuery("select @@global.tidb_ddl_error_count_limit") + res.Check(testkit.Rows("100")) +} + +func TestLoadDDLDistributeVars(t *testing.T) { + store := testkit.CreateMockStore(t) + tk := testkit.NewTestKit(t, store) + tk.MustExec("use test") + require.Equal(t, variable.DefTiDBEnableDistTask, disttask.TiDBEnableDistTask) + + tk.MustGetDBError("set @@global.tidb_enable_dist_task = invalid_val", variable.ErrWrongValueForVar) + require.Equal(t, disttask.TiDBEnableDistTask, variable.EnableDistTask.Load()) + tk.MustExec("set @@global.tidb_enable_dist_task = 'on'") + require.Equal(t, true, variable.EnableDistTask.Load()) + tk.MustExec(fmt.Sprintf("set @@global.tidb_enable_dist_task = %v", disttask.TiDBEnableDistTask)) + require.Equal(t, disttask.TiDBEnableDistTask, variable.EnableDistTask.Load()) +} + +// this test will change the fail-point `mockAutoIDChange`, so we move it to the `testRecoverTable` suite +func TestRenameTable(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/meta/autoid/mockAutoIDChange", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/meta/autoid/mockAutoIDChange")) + }() + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("drop database if exists rename1") + tk.MustExec("drop database if exists rename2") + tk.MustExec("drop database if exists rename3") + + tk.MustExec("create database rename1") + tk.MustExec("create database rename2") + tk.MustExec("create database rename3") + tk.MustExec("create table rename1.t (a int primary key auto_increment)") + tk.MustExec("insert rename1.t values ()") + tk.MustExec("rename table rename1.t to rename2.t") + // Make sure the drop old database doesn't affect the rename3.t's operations. + tk.MustExec("drop database rename1") + tk.MustExec("insert rename2.t values ()") + tk.MustExec("rename table rename2.t to rename3.t") + tk.MustExec("insert rename3.t values ()") + tk.MustQuery("select * from rename3.t").Check(testkit.Rows("1", "2", "3")) + // Make sure the drop old database doesn't affect the rename3.t's operations. + tk.MustExec("drop database rename2") + tk.MustExec("insert rename3.t values ()") + tk.MustQuery("select * from rename3.t").Check(testkit.Rows("1", "2", "3", "4")) + tk.MustExec("drop database rename3") + + tk.MustExec("create database rename1") + tk.MustExec("create database rename2") + tk.MustExec("create table rename1.t (a int primary key auto_increment)") + tk.MustExec("rename table rename1.t to rename2.t1") + tk.MustExec("insert rename2.t1 values ()") + result := tk.MustQuery("select * from rename2.t1") + result.Check(testkit.Rows("1")) + // Make sure the drop old database doesn't affect the t1's operations. + tk.MustExec("drop database rename1") + tk.MustExec("insert rename2.t1 values ()") + result = tk.MustQuery("select * from rename2.t1") + result.Check(testkit.Rows("1", "2")) + // Rename a table to another table in the same database. + tk.MustExec("rename table rename2.t1 to rename2.t2") + tk.MustExec("insert rename2.t2 values ()") + result = tk.MustQuery("select * from rename2.t2") + result.Check(testkit.Rows("1", "2", "3")) + tk.MustExec("drop database rename2") + + tk.MustExec("create database rename1") + tk.MustExec("create database rename2") + tk.MustExec("create table rename1.t (a int primary key auto_increment)") + tk.MustExec("insert rename1.t values ()") + tk.MustExec("rename table rename1.t to rename2.t1") + // Make sure the value is greater than autoid.step. + tk.MustExec("insert rename2.t1 values (100000)") + tk.MustExec("insert rename2.t1 values ()") + result = tk.MustQuery("select * from rename2.t1") + result.Check(testkit.Rows("1", "100000", "100001")) + tk.MustExecToErr("insert rename1.t values ()") + tk.MustExec("drop database rename1") + tk.MustExec("drop database rename2") +} + +func TestRenameMultiTables(t *testing.T) { + require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/meta/autoid/mockAutoIDChange", `return(true)`)) + defer func() { + require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/meta/autoid/mockAutoIDChange")) + }() + store := testkit.CreateMockStore(t, mockstore.WithDDLChecker()) + + tk := testkit.NewTestKit(t, store) + + tk.MustExec("drop database if exists rename1") + tk.MustExec("drop database if exists rename2") + tk.MustExec("drop database if exists rename3") + tk.MustExec("drop database if exists rename4") + + tk.MustExec("create database rename1") + tk.MustExec("create database rename2") + tk.MustExec("create database rename3") + tk.MustExec("create database rename4") + tk.MustExec("create table rename1.t1 (a int primary key auto_increment)") + tk.MustExec("create table rename3.t3 (a int primary key auto_increment)") + tk.MustExec("insert rename1.t1 values ()") + tk.MustExec("insert rename3.t3 values ()") + tk.MustExec("rename table rename1.t1 to rename2.t2, rename3.t3 to rename4.t4") + // Make sure the drop old database doesn't affect t2,t4's operations. + tk.MustExec("drop database rename1") + tk.MustExec("insert rename2.t2 values ()") + tk.MustExec("drop database rename3") + tk.MustExec("insert rename4.t4 values ()") + tk.MustQuery("select * from rename2.t2").Check(testkit.Rows("1", "2")) + tk.MustQuery("select * from rename4.t4").Check(testkit.Rows("1", "2")) + // Rename a table to another table in the same database. + tk.MustExec("rename table rename2.t2 to rename2.t1, rename4.t4 to rename4.t3") + tk.MustExec("insert rename2.t1 values ()") + tk.MustQuery("select * from rename2.t1").Check(testkit.Rows("1", "2", "3")) + tk.MustExec("insert rename4.t3 values ()") + tk.MustQuery("select * from rename4.t3").Check(testkit.Rows("1", "2", "3")) + tk.MustExec("drop database rename2") + tk.MustExec("drop database rename4") + + tk.MustExec("create database rename1") + tk.MustExec("create database rename2") + tk.MustExec("create database rename3") + tk.MustExec("create table rename1.t1 (a int primary key auto_increment)") + tk.MustExec("create table rename3.t3 (a int primary key auto_increment)") + tk.MustGetErrCode("rename table rename1.t1 to rename2.t2, rename3.t3 to rename2.t2", errno.ErrTableExists) + tk.MustExec("rename table rename1.t1 to rename2.t2, rename2.t2 to rename1.t1") + tk.MustExec("rename table rename1.t1 to rename2.t2, rename3.t3 to rename1.t1") + tk.MustExec("use rename1") + tk.MustQuery("show tables").Check(testkit.Rows("t1")) + tk.MustExec("use rename2") + tk.MustQuery("show tables").Check(testkit.Rows("t2")) + tk.MustExec("use rename3") + tk.MustExec("create table rename3.t3 (a int primary key auto_increment)") + tk.MustGetErrCode("rename table rename1.t1 to rename1.t2, rename1.t1 to rename3.t3", errno.ErrTableExists) + tk.MustGetErrCode("rename table rename1.t1 to rename1.t2, rename1.t1 to rename3.t4", errno.ErrNoSuchTable) + tk.MustExec("drop database rename1") + tk.MustExec("drop database rename2") + tk.MustExec("drop database rename3") +} diff --git a/pkg/meta/BUILD.bazel b/pkg/meta/BUILD.bazel new file mode 100644 index 0000000000000..2ade72bba8893 --- /dev/null +++ b/pkg/meta/BUILD.bazel @@ -0,0 +1,46 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "meta", + srcs = [ + "meta.go", + "meta_autoid.go", + ], + importpath = "github.com/pingcap/tidb/pkg/meta", + visibility = ["//visibility:public"], + deps = [ + "//pkg/domain/resourcegroup", + "//pkg/errno", + "//pkg/kv", + "//pkg/metrics", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/structure", + "//pkg/util/dbterror", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_kvproto//pkg/kvrpcpb", + ], +) + +go_test( + name = "meta_test", + timeout = "short", + srcs = [ + "main_test.go", + "meta_test.go", + ], + embed = [":meta"], + flaky = True, + shard_count = 11, + deps = [ + "//pkg/kv", + "//pkg/parser/ast", + "//pkg/parser/model", + "//pkg/store/mockstore", + "//pkg/testkit/testsetup", + "//pkg/util", + "@com_github_pingcap_errors//:errors", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + ], +) diff --git a/tests/integrationtest/r/ddl/db_integration.result b/tests/integrationtest/r/ddl/db_integration.result new file mode 100644 index 0000000000000..ef71f0c8f7f69 --- /dev/null +++ b/tests/integrationtest/r/ddl/db_integration.result @@ -0,0 +1,1293 @@ +drop table if exists test_zero_date; +set session sql_mode='STRICT_TRANS_TABLES,NO_ZERO_DATE,NO_ENGINE_SUBSTITUTION'; +create table test_zero_date(agent_start_time date NOT NULL DEFAULT '0000-00-00'); +Error 1067 (42000): Invalid default value for 'agent_start_time' +create table test_zero_date(agent_start_time datetime NOT NULL DEFAULT '0000-00-00 00:00:00'); +Error 1067 (42000): Invalid default value for 'agent_start_time' +create table test_zero_date(agent_start_time timestamp NOT NULL DEFAULT '0000-00-00 00:00:00'); +Error 1067 (42000): Invalid default value for 'agent_start_time' +create table test_zero_date(a timestamp default '0000-00-00 00'); +Error 1067 (42000): Invalid default value for 'a' +create table test_zero_date(a timestamp default 0); +Error 1067 (42000): Invalid default value for 'a' +set session sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +create table test_zero_date (a timestamp default 0); +insert into test_zero_date values (0); +select a, unix_timestamp(a) from test_zero_date; +a unix_timestamp(a) +0000-00-00 00:00:00 0 +update test_zero_date set a = '2001-01-01 11:11:11' where a = 0; +replace into test_zero_date values (0); +delete from test_zero_date where a = 0; +update test_zero_date set a = 0 where a = '2001-01-01 11:11:11'; +set session sql_mode='ONLY_FULL_GROUP_BY,STRICT_TRANS_TABLES,NO_ZERO_IN_DATE,NO_ZERO_DATE,ERROR_FOR_DIVISION_BY_ZERO,NO_AUTO_CREATE_USER,NO_ENGINE_SUBSTITUTION'; +insert into test_zero_date values (0); +Error 1292 (22007): Incorrect timestamp value: '0' for column 'a' at row 1 +replace into test_zero_date values (0); +Error 1292 (22007): Incorrect timestamp value: '0' for column 'a' at row 1 +update test_zero_date set a = 0 where a = 0; +Error 1292 (22007): Incorrect timestamp value: '0' +delete from test_zero_date where a = 0; +select a, unix_timestamp(a) from test_zero_date; +a unix_timestamp(a) +drop table test_zero_date; +set session sql_mode=''; +create table test_zero_date (a timestamp default 0); +drop table test_zero_date; +create table test_zero_date (a int); +insert into test_zero_date values (0); +alter table test_zero_date modify a date; +set session sql_mode='NO_ZERO_DATE'; +drop table test_zero_date; +create table test_zero_date (a timestamp default 0); +drop table test_zero_date; +create table test_zero_date (a int); +insert into test_zero_date values (0); +alter table test_zero_date modify a date; +set session sql_mode='STRICT_TRANS_TABLES'; +drop table test_zero_date; +create table test_zero_date (a timestamp default 0); +drop table test_zero_date; +create table test_zero_date (a int); +insert into test_zero_date values (0); +alter table test_zero_date modify a date; +Error 1292 (22007): Truncated incorrect date value: '0' +set session sql_mode='NO_ZERO_DATE,STRICT_TRANS_TABLES'; +drop table test_zero_date; +create table test_zero_date (a timestamp default 0); +Error 1067 (42000): Invalid default value for 'a' +create table test_zero_date (a int); +insert into test_zero_date values (0); +alter table test_zero_date modify a date; +Error 1292 (22007): Truncated incorrect date value: '0' +drop table if exists test_zero_date; +set session sql_mode=default; +drop table if exists t; +create table t(c1 decimal default 1.7976931348623157E308); +Error 1067 (42000): Invalid default value for 'c1' +create table t( c1 varchar(2) default 'TiDB'); +Error 1067 (42000): Invalid default value for 'c1' +drop table if exists t_without_length; +create table t_without_length (a text primary key); +Error 1170 (42000): BLOB/TEXT column 'a' used in key specification without a key length +drop table if exists t; +create table t(xxx.t.a bigint); +Error 1102 (42000): Incorrect database name 'xxx' +create table t(ddl__db_integration.tttt.a bigint); +Error 1103 (42000): Incorrect table name 'tttt' +create table t(t.tttt.a bigint); +Error 1102 (42000): Incorrect database name 't' +drop table if exists t1; +create table t1 (quantity decimal(2) unsigned); +insert into t1 values (500), (-500), (~0), (-1); +Error 1264 (22003): Out of range value for column 'quantity' at row 1 +drop table t1; +set sql_mode=''; +create table t1 (quantity decimal(2) unsigned); +insert into t1 values (500), (-500), (~0), (-1); +select * from t1; +quantity +99 +0 +99 +0 +drop table t1; +set sql_mode=default; +drop table if exists issue3833, issue3833_2; +create table issue3833 (b char(0), c binary(0), d varchar(0)); +create index idx on issue3833 (b); +Error 1167 (42000): The used storage engine can't index column 'b' +alter table issue3833 add index idx (b); +Error 1167 (42000): The used storage engine can't index column 'b' +create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(b)); +Error 1167 (42000): The used storage engine can't index column 'b' +create index idx on issue3833 (c); +Error 1167 (42000): The used storage engine can't index column 'c' +alter table issue3833 add index idx (c); +Error 1167 (42000): The used storage engine can't index column 'c' +create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(c)); +Error 1167 (42000): The used storage engine can't index column 'c' +create index idx on issue3833 (d); +Error 1167 (42000): The used storage engine can't index column 'd' +alter table issue3833 add index idx (d); +Error 1167 (42000): The used storage engine can't index column 'd' +create table issue3833_2 (b char(0), c binary(0), d varchar(0), index(d)); +Error 1167 (42000): The used storage engine can't index column 'd' +drop table if exists test_error_code_succ, test_error_code1, test_error_code_2, test_error_code_3, test_error_code_null, test_error_code_succ; +drop table if exists t1, t2, test_add_columns_on_update, test_drop_column, test_drop_columns; +create database aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa; +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +create database test; +Error 1007 (HY000): Can't create database 'test'; database exists +create database test1 character set uft8; +Error 1115 (42000): Unknown character set: 'uft8' +create database test2 character set gkb; +Error 1115 (42000): Unknown character set: 'gkb' +create database test3 character set laitn1; +Error 1115 (42000): Unknown character set: 'laitn1' +drop database db_not_exist; +Error 1008 (HY000): Can't drop database 'db_not_exist'; database doesn't exist +create table test_error_code_succ (c1 int, c2 int, c3 int, primary key(c3)); +create table test_error_code_succ (c1 int, c2 int, c3 int); +Error 1050 (42S01): Table 'ddl__db_integration.test_error_code_succ' already exists +create table test_error_code1 (c1 int, c2 int, c2 int); +Error 1060 (42S21): Duplicate column name 'c2' +create table test_error_code1 (c1 int, aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa int); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +create table test_error_code1 (c1 int, `_tidb_rowid` int); +Error 1166 (42000): Incorrect column name '_tidb_rowid' +create table aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa(a int); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +create table test_error_code1 (c1 int, c2 int, key aa (c1, c2), key aa (c1)); +Error 1061 (42000): Duplicate key name 'aa' +create table test_error_code1 (c1 int, c2 int, c3 int, key(c_not_exist)); +Error 1072 (42000): column does not exist: c_not_exist +create table test_error_code1 (c1 int, c2 int, c3 int, primary key(c_not_exist)); +Error 1072 (42000): Key column 'c_not_exist' doesn't exist in table +create table test_error_code1 (c1 int not null default ''); +Error 1067 (42000): Invalid default value for 'c1' +CREATE TABLE `t` (`a` double DEFAULT 1.0 DEFAULT 2.0 DEFAULT now()); +Error 1067 (42000): Invalid default value for 'a' +CREATE TABLE `t` (`a` double DEFAULT now()); +Error 1067 (42000): Invalid default value for 'a' +create table t1(a int) character set uft8; +Error 1115 (42000): Unknown character set: 'uft8' +create table t1(a int) character set gkb; +Error 1115 (42000): Unknown character set: 'gkb' +create table t1(a int) character set laitn1; +Error 1115 (42000): Unknown character set: 'laitn1' +create table test_error_code (a int not null ,b int not null,c int not null, d int not null, foreign key (b, c) references product(id)); +Error 1239 (42000): Incorrect foreign key definition for 'fk_1': Key reference and table reference don't match +create table test_error_code_2; +Error 1113 (42000): A table must have at least 1 column +create table test_error_code_2 (unique(c1)); +Error 1113 (42000): A table must have at least 1 column +create table test_error_code_2(c1 int, c2 int, c3 int, primary key(c1), primary key(c2)); +Error 1068 (42000): Multiple primary key defined +create table test_error_code_3(pt blob ,primary key (pt)); +Error 1170 (42000): BLOB/TEXT column 'pt' used in key specification without a key length +create table test_error_code_3(a text, unique (a(769))); +Error 1071 (42000): Specified key was too long (3076 bytes); max key length is 3072 bytes +create table test_error_code_3(a text charset ascii, unique (a(3073))); +Error 1071 (42000): Specified key was too long (3073 bytes); max key length is 3072 bytes +create table test_error_code_3(`id` int, key `primary`(`id`)); +Error 1280 (42000): Incorrect index name 'primary' +create table t2(c1.c2 blob default null); +Error 1103 (42000): Incorrect table name 'c1' +create table t2 (id int default null primary key , age int); +Error 1067 (42000): Invalid default value for 'id' +create table t2 (id int null primary key , age int); +Error 1171 (42000): All parts of a PRIMARY KEY must be NOT NULL; if you need NULL in a key, use UNIQUE instead +create table t2 (id int default null, age int, primary key(id)); +Error 1171 (42000): All parts of a PRIMARY KEY must be NOT NULL; if you need NULL in a key, use UNIQUE instead +create table t2 (id int null, age int, primary key(id)); +Error 1171 (42000): All parts of a PRIMARY KEY must be NOT NULL; if you need NULL in a key, use UNIQUE instead +create table t2 (id int auto_increment, c int auto_increment); +Error 1075 (42000): Incorrect table definition; there can be only one auto column and it must be defined as a key +create table t2 (a datetime(2) default current_timestamp(3)); +Error 1067 (42000): Invalid default value for 'a' +create table t2 (a datetime(2) default current_timestamp(2) on update current_timestamp); +Error 1294 (HY000): Invalid ON UPDATE clause for 'a' column +create table t2 (a datetime default current_timestamp on update current_timestamp(2)); +Error 1294 (HY000): Invalid ON UPDATE clause for 'a' column +create table t2 (a datetime(2) default current_timestamp(2) on update current_timestamp(3)); +Error 1294 (HY000): Invalid ON UPDATE clause for 'a' column +create table t(a blob(10), index(a(0))); +Error 1391 (HY000): Key part 'a' length cannot be 0 +create table t(a char(10), index(a(0))); +Error 1391 (HY000): Key part 'a' length cannot be 0 +create table t2 (id int primary key , age int); +alter table test_error_code_succ add column c1 int; +Error 1060 (42S21): Duplicate column name 'c1' +alter table test_error_code_succ add column aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa int; +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +alter table test_comment comment 'test comment'; +Error 1146 (42S02): Table 'ddl__db_integration.test_comment' doesn't exist +alter table test_error_code_succ add column `a ` int ; +Error 1166 (42000): Incorrect column name 'a ' +alter table test_error_code_succ add column `_tidb_rowid` int ; +Error 1166 (42000): Incorrect column name '_tidb_rowid' +create table test_on_update (c1 int, c2 int); +alter table test_on_update add column c3 int on update current_timestamp; +Error 1294 (HY000): Invalid ON UPDATE clause for 'c3' column +create table test_on_update_2(c int on update current_timestamp); +Error 1294 (HY000): Invalid ON UPDATE clause for 'c' column +alter table test_error_code_succ add column c1 int, add column c1 int; +Error 1060 (42S21): Duplicate column name 'c1' +alter table test_error_code_succ add column (aa int, aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa int); +Error 1059 (42000): Identifier name 'aaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaaa' is too long +alter table test_error_code_succ add column `a ` int, add column `b ` int; +Error 1166 (42000): Incorrect column name 'a ' +create table test_add_columns_on_update (c1 int, c2 int); +alter table test_add_columns_on_update add column cc int, add column c3 int on update current_timestamp; +Error 1294 (HY000): Invalid ON UPDATE clause for 'c3' column +alter table test_error_code_succ drop c_not_exist; +Error 1091 (42000): Can't DROP 'c_not_exist'; check that column/key exists +create table test_drop_column (c1 int ); +alter table test_drop_column drop column c1; +Error 1090 (42000): can't drop only column c1 in table test_drop_column +alter table test_error_code_succ drop c_not_exist, drop cc_not_exist; +Error 1091 (42000): Can't DROP 'c_not_exist'; check that column/key exists +create table test_drop_columns (c1 int); +alter table test_drop_columns add column c2 int first, add column c3 int after c1; +alter table test_drop_columns drop column c1, drop column c2, drop column c3; +Error 1090 (42000): You can't delete all columns with ALTER TABLE; use DROP TABLE instead +alter table test_drop_columns drop column c1, add column c2 int; +Error 1060 (42S21): Duplicate column name 'c2' +alter table test_drop_columns drop column c1, drop column c1; +Error 8200 (HY000): Unsupported operate same column 'c1' +alter table test_error_code_succ add index idx (c_not_exist); +Error 1072 (42000): column does not exist: c_not_exist +alter table test_error_code_succ add index idx (c1); +alter table test_error_code_succ add index idx (c1); +Error 1061 (42000): Duplicate key name 'idx' +alter table test_error_code_succ drop index idx_not_exist; +Error 1091 (42000): index idx_not_exist doesn't exist +alter table test_error_code_succ drop column c3; +Error 8200 (HY000): Unsupported drop integer primary key +alter table test_error_code_succ modify testx.test_error_code_succ.c1 bigint; +Error 1102 (42000): Incorrect database name 'testx' +alter table test_error_code_succ modify t.c1 bigint; +Error 1103 (42000): Incorrect table name 't' +alter table test_error_code_succ change c1 _tidb_rowid bigint; +Error 1166 (42000): Incorrect column name '_tidb_rowid' +alter table test_error_code_succ rename column c1 to _tidb_rowid; +Error 1166 (42000): Incorrect column name '_tidb_rowid' +create table test_error_code_null(c1 char(100) not null); +insert into test_error_code_null (c1) values(null); +Error 1048 (23000): Column 'c1' cannot be null +drop table if exists t; +create table t (a decimal(1, 2)); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'a'). +create table t (a float(1, 2)); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'a'). +create table t (a double(1, 2)); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'a'). +create table t (a double(1, 1)); +alter table t add column b decimal(1, 2); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'b'). +alter table t modify column a float(1, 4); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'a'). +alter table t change column a aa float(1, 4); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'aa'). +drop table t; +drop table if exists t_too_many_indexes; +create table t_too_many_indexes (c0 int,c1 int,c2 int,c3 int,c4 int,c5 int,c6 int,c7 int,c8 int,c9 int,c10 int,c11 int,c12 int,c13 int,c14 int,c15 int,c16 int,c17 int,c18 int,c19 int,c20 int,c21 int,c22 int,c23 int,c24 int,c25 int,c26 int,c27 int,c28 int,c29 int,c30 int,c31 int,c32 int,c33 int,c34 int,c35 int,c36 int,c37 int,c38 int,c39 int,c40 int,c41 int,c42 int,c43 int,c44 int,c45 int,c46 int,c47 int,c48 int,c49 int,c50 int,c51 int,c52 int,c53 int,c54 int,c55 int,c56 int,c57 int,c58 int,c59 int,c60 int,c61 int,c62 int,c63 int,c64 int,c65 int,c66 int,c67 int,c68 int,c69 int,c70 int,c71 int,c72 int,c73 int,c74 int,c75 int,c76 int,c77 int,c78 int,c79 int,c80 int,c81 int,c82 int,c83 int,c84 int,c85 int,c86 int,c87 int,c88 int,c89 int,c90 int,c91 int,c92 int,c93 int,c94 int,c95 int,c96 int,c97 int,c98 int,c99 int,key k0(c0),key k1(c1),key k2(c2),key k3(c3),key k4(c4),key k5(c5),key k6(c6),key k7(c7),key k8(c8),key k9(c9),key k10(c10),key k11(c11),key k12(c12),key k13(c13),key k14(c14),key k15(c15),key k16(c16),key k17(c17),key k18(c18),key k19(c19),key k20(c20),key k21(c21),key k22(c22),key k23(c23),key k24(c24),key k25(c25),key k26(c26),key k27(c27),key k28(c28),key k29(c29),key k30(c30),key k31(c31),key k32(c32),key k33(c33),key k34(c34),key k35(c35),key k36(c36),key k37(c37),key k38(c38),key k39(c39),key k40(c40),key k41(c41),key k42(c42),key k43(c43),key k44(c44),key k45(c45),key k46(c46),key k47(c47),key k48(c48),key k49(c49),key k50(c50),key k51(c51),key k52(c52),key k53(c53),key k54(c54),key k55(c55),key k56(c56),key k57(c57),key k58(c58),key k59(c59),key k60(c60),key k61(c61),key k62(c62),key k63(c63),key k64(c64),key k65(c65),key k66(c66),key k67(c67),key k68(c68),key k69(c69),key k70(c70),key k71(c71),key k72(c72),key k73(c73),key k74(c74),key k75(c75),key k76(c76),key k77(c77),key k78(c78),key k79(c79),key k80(c80),key k81(c81),key k82(c82),key k83(c83),key k84(c84),key k85(c85),key k86(c86),key k87(c87),key k88(c88),key k89(c89),key k90(c90),key k91(c91),key k92(c92),key k93(c93),key k94(c94),key k95(c95),key k96(c96),key k97(c97),key k98(c98),key k99(c99)); +Error 1069 (42000): Too many keys specified; max 64 keys allowed +drop table if exists t_too_many; +create table t_index_too_many (c0 int,c1 int,c2 int,c3 int,c4 int,c5 int,c6 int,c7 int,c8 int,c9 int,c10 int,c11 int,c12 int,c13 int,c14 int,c15 int,c16 int,c17 int,c18 int,c19 int,c20 int,c21 int,c22 int,c23 int,c24 int,c25 int,c26 int,c27 int,c28 int,c29 int,c30 int,c31 int,c32 int,c33 int,c34 int,c35 int,c36 int,c37 int,c38 int,c39 int,c40 int,c41 int,c42 int,c43 int,c44 int,c45 int,c46 int,c47 int,c48 int,c49 int,c50 int,c51 int,c52 int,c53 int,c54 int,c55 int,c56 int,c57 int,c58 int,c59 int,c60 int,c61 int,c62 int,c63 int,c64 int,c65 int,c66 int,c67 int,c68 int,c69 int,c70 int,c71 int,c72 int,c73 int,c74 int,c75 int,c76 int,c77 int,c78 int,c79 int,c80 int,c81 int,c82 int,c83 int,c84 int,c85 int,c86 int,c87 int,c88 int,c89 int,c90 int,c91 int,c92 int,c93 int,c94 int,c95 int,c96 int,c97 int,c98 int,c99 int,key k0(c0),key k1(c1),key k2(c2),key k3(c3),key k4(c4),key k5(c5),key k6(c6),key k7(c7),key k8(c8),key k9(c9),key k10(c10),key k11(c11),key k12(c12),key k13(c13),key k14(c14),key k15(c15),key k16(c16),key k17(c17),key k18(c18),key k19(c19),key k20(c20),key k21(c21),key k22(c22),key k23(c23),key k24(c24),key k25(c25),key k26(c26),key k27(c27),key k28(c28),key k29(c29),key k30(c30),key k31(c31),key k32(c32),key k33(c33),key k34(c34),key k35(c35),key k36(c36),key k37(c37),key k38(c38),key k39(c39),key k40(c40),key k41(c41),key k42(c42),key k43(c43),key k44(c44),key k45(c45),key k46(c46),key k47(c47),key k48(c48),key k49(c49),key k50(c50),key k51(c51),key k52(c52),key k53(c53),key k54(c54),key k55(c55),key k56(c56),key k57(c57),key k58(c58),key k59(c59),key k60(c60),key k61(c61),key k62(c62)); +create index idx1 on t_index_too_many (c62); +create index idx2 on t_index_too_many (c63); +Error 1069 (42000): Too many keys specified; max 64 keys allowed +drop table if exists t_column_too_many; +create table t_column_too_many (a0 int,a1 int,a2 int,a3 int,a4 int,a5 int,a6 int,a7 int,a8 int,a9 int,a10 int,a11 int,a12 int,a13 int,a14 int,a15 int,a16 int,a17 int,a18 int,a19 int,a20 int,a21 int,a22 int,a23 int,a24 int,a25 int,a26 int,a27 int,a28 int,a29 int,a30 int,a31 int,a32 int,a33 int,a34 int,a35 int,a36 int,a37 int,a38 int,a39 int,a40 int,a41 int,a42 int,a43 int,a44 int,a45 int,a46 int,a47 int,a48 int,a49 int,a50 int,a51 int,a52 int,a53 int,a54 int,a55 int,a56 int,a57 int,a58 int,a59 int,a60 int,a61 int,a62 int,a63 int,a64 int,a65 int,a66 int,a67 int,a68 int,a69 int,a70 int,a71 int,a72 int,a73 int,a74 int,a75 int,a76 int,a77 int,a78 int,a79 int,a80 int,a81 int,a82 int,a83 int,a84 int,a85 int,a86 int,a87 int,a88 int,a89 int,a90 int,a91 int,a92 int,a93 int,a94 int,a95 int,a96 int,a97 int,a98 int,a99 int,a100 int,a101 int,a102 int,a103 int,a104 int,a105 int,a106 int,a107 int,a108 int,a109 int,a110 int,a111 int,a112 int,a113 int,a114 int,a115 int,a116 int,a117 int,a118 int,a119 int,a120 int,a121 int,a122 int,a123 int,a124 int,a125 int,a126 int,a127 int,a128 int,a129 int,a130 int,a131 int,a132 int,a133 int,a134 int,a135 int,a136 int,a137 int,a138 int,a139 int,a140 int,a141 int,a142 int,a143 int,a144 int,a145 int,a146 int,a147 int,a148 int,a149 int,a150 int,a151 int,a152 int,a153 int,a154 int,a155 int,a156 int,a157 int,a158 int,a159 int,a160 int,a161 int,a162 int,a163 int,a164 int,a165 int,a166 int,a167 int,a168 int,a169 int,a170 int,a171 int,a172 int,a173 int,a174 int,a175 int,a176 int,a177 int,a178 int,a179 int,a180 int,a181 int,a182 int,a183 int,a184 int,a185 int,a186 int,a187 int,a188 int,a189 int,a190 int,a191 int,a192 int,a193 int,a194 int,a195 int,a196 int,a197 int,a198 int,a199 int,a200 int,a201 int,a202 int,a203 int,a204 int,a205 int,a206 int,a207 int,a208 int,a209 int,a210 int,a211 int,a212 int,a213 int,a214 int,a215 int,a216 int,a217 int,a218 int,a219 int,a220 int,a221 int,a222 int,a223 int,a224 int,a225 int,a226 int,a227 int,a228 int,a229 int,a230 int,a231 int,a232 int,a233 int,a234 int,a235 int,a236 int,a237 int,a238 int,a239 int,a240 int,a241 int,a242 int,a243 int,a244 int,a245 int,a246 int,a247 int,a248 int,a249 int,a250 int,a251 int,a252 int,a253 int,a254 int,a255 int,a256 int,a257 int,a258 int,a259 int,a260 int,a261 int,a262 int,a263 int,a264 int,a265 int,a266 int,a267 int,a268 int,a269 int,a270 int,a271 int,a272 int,a273 int,a274 int,a275 int,a276 int,a277 int,a278 int,a279 int,a280 int,a281 int,a282 int,a283 int,a284 int,a285 int,a286 int,a287 int,a288 int,a289 int,a290 int,a291 int,a292 int,a293 int,a294 int,a295 int,a296 int,a297 int,a298 int,a299 int,a300 int,a301 int,a302 int,a303 int,a304 int,a305 int,a306 int,a307 int,a308 int,a309 int,a310 int,a311 int,a312 int,a313 int,a314 int,a315 int,a316 int,a317 int,a318 int,a319 int,a320 int,a321 int,a322 int,a323 int,a324 int,a325 int,a326 int,a327 int,a328 int,a329 int,a330 int,a331 int,a332 int,a333 int,a334 int,a335 int,a336 int,a337 int,a338 int,a339 int,a340 int,a341 int,a342 int,a343 int,a344 int,a345 int,a346 int,a347 int,a348 int,a349 int,a350 int,a351 int,a352 int,a353 int,a354 int,a355 int,a356 int,a357 int,a358 int,a359 int,a360 int,a361 int,a362 int,a363 int,a364 int,a365 int,a366 int,a367 int,a368 int,a369 int,a370 int,a371 int,a372 int,a373 int,a374 int,a375 int,a376 int,a377 int,a378 int,a379 int,a380 int,a381 int,a382 int,a383 int,a384 int,a385 int,a386 int,a387 int,a388 int,a389 int,a390 int,a391 int,a392 int,a393 int,a394 int,a395 int,a396 int,a397 int,a398 int,a399 int,a400 int,a401 int,a402 int,a403 int,a404 int,a405 int,a406 int,a407 int,a408 int,a409 int,a410 int,a411 int,a412 int,a413 int,a414 int,a415 int,a416 int,a417 int,a418 int,a419 int,a420 int,a421 int,a422 int,a423 int,a424 int,a425 int,a426 int,a427 int,a428 int,a429 int,a430 int,a431 int,a432 int,a433 int,a434 int,a435 int,a436 int,a437 int,a438 int,a439 int,a440 int,a441 int,a442 int,a443 int,a444 int,a445 int,a446 int,a447 int,a448 int,a449 int,a450 int,a451 int,a452 int,a453 int,a454 int,a455 int,a456 int,a457 int,a458 int,a459 int,a460 int,a461 int,a462 int,a463 int,a464 int,a465 int,a466 int,a467 int,a468 int,a469 int,a470 int,a471 int,a472 int,a473 int,a474 int,a475 int,a476 int,a477 int,a478 int,a479 int,a480 int,a481 int,a482 int,a483 int,a484 int,a485 int,a486 int,a487 int,a488 int,a489 int,a490 int,a491 int,a492 int,a493 int,a494 int,a495 int,a496 int,a497 int,a498 int,a499 int,a500 int,a501 int,a502 int,a503 int,a504 int,a505 int,a506 int,a507 int,a508 int,a509 int,a510 int,a511 int,a512 int,a513 int,a514 int,a515 int,a516 int,a517 int,a518 int,a519 int,a520 int,a521 int,a522 int,a523 int,a524 int,a525 int,a526 int,a527 int,a528 int,a529 int,a530 int,a531 int,a532 int,a533 int,a534 int,a535 int,a536 int,a537 int,a538 int,a539 int,a540 int,a541 int,a542 int,a543 int,a544 int,a545 int,a546 int,a547 int,a548 int,a549 int,a550 int,a551 int,a552 int,a553 int,a554 int,a555 int,a556 int,a557 int,a558 int,a559 int,a560 int,a561 int,a562 int,a563 int,a564 int,a565 int,a566 int,a567 int,a568 int,a569 int,a570 int,a571 int,a572 int,a573 int,a574 int,a575 int,a576 int,a577 int,a578 int,a579 int,a580 int,a581 int,a582 int,a583 int,a584 int,a585 int,a586 int,a587 int,a588 int,a589 int,a590 int,a591 int,a592 int,a593 int,a594 int,a595 int,a596 int,a597 int,a598 int,a599 int,a600 int,a601 int,a602 int,a603 int,a604 int,a605 int,a606 int,a607 int,a608 int,a609 int,a610 int,a611 int,a612 int,a613 int,a614 int,a615 int,a616 int,a617 int,a618 int,a619 int,a620 int,a621 int,a622 int,a623 int,a624 int,a625 int,a626 int,a627 int,a628 int,a629 int,a630 int,a631 int,a632 int,a633 int,a634 int,a635 int,a636 int,a637 int,a638 int,a639 int,a640 int,a641 int,a642 int,a643 int,a644 int,a645 int,a646 int,a647 int,a648 int,a649 int,a650 int,a651 int,a652 int,a653 int,a654 int,a655 int,a656 int,a657 int,a658 int,a659 int,a660 int,a661 int,a662 int,a663 int,a664 int,a665 int,a666 int,a667 int,a668 int,a669 int,a670 int,a671 int,a672 int,a673 int,a674 int,a675 int,a676 int,a677 int,a678 int,a679 int,a680 int,a681 int,a682 int,a683 int,a684 int,a685 int,a686 int,a687 int,a688 int,a689 int,a690 int,a691 int,a692 int,a693 int,a694 int,a695 int,a696 int,a697 int,a698 int,a699 int,a700 int,a701 int,a702 int,a703 int,a704 int,a705 int,a706 int,a707 int,a708 int,a709 int,a710 int,a711 int,a712 int,a713 int,a714 int,a715 int,a716 int,a717 int,a718 int,a719 int,a720 int,a721 int,a722 int,a723 int,a724 int,a725 int,a726 int,a727 int,a728 int,a729 int,a730 int,a731 int,a732 int,a733 int,a734 int,a735 int,a736 int,a737 int,a738 int,a739 int,a740 int,a741 int,a742 int,a743 int,a744 int,a745 int,a746 int,a747 int,a748 int,a749 int,a750 int,a751 int,a752 int,a753 int,a754 int,a755 int,a756 int,a757 int,a758 int,a759 int,a760 int,a761 int,a762 int,a763 int,a764 int,a765 int,a766 int,a767 int,a768 int,a769 int,a770 int,a771 int,a772 int,a773 int,a774 int,a775 int,a776 int,a777 int,a778 int,a779 int,a780 int,a781 int,a782 int,a783 int,a784 int,a785 int,a786 int,a787 int,a788 int,a789 int,a790 int,a791 int,a792 int,a793 int,a794 int,a795 int,a796 int,a797 int,a798 int,a799 int,a800 int,a801 int,a802 int,a803 int,a804 int,a805 int,a806 int,a807 int,a808 int,a809 int,a810 int,a811 int,a812 int,a813 int,a814 int,a815 int,a816 int,a817 int,a818 int,a819 int,a820 int,a821 int,a822 int,a823 int,a824 int,a825 int,a826 int,a827 int,a828 int,a829 int,a830 int,a831 int,a832 int,a833 int,a834 int,a835 int,a836 int,a837 int,a838 int,a839 int,a840 int,a841 int,a842 int,a843 int,a844 int,a845 int,a846 int,a847 int,a848 int,a849 int,a850 int,a851 int,a852 int,a853 int,a854 int,a855 int,a856 int,a857 int,a858 int,a859 int,a860 int,a861 int,a862 int,a863 int,a864 int,a865 int,a866 int,a867 int,a868 int,a869 int,a870 int,a871 int,a872 int,a873 int,a874 int,a875 int,a876 int,a877 int,a878 int,a879 int,a880 int,a881 int,a882 int,a883 int,a884 int,a885 int,a886 int,a887 int,a888 int,a889 int,a890 int,a891 int,a892 int,a893 int,a894 int,a895 int,a896 int,a897 int,a898 int,a899 int,a900 int,a901 int,a902 int,a903 int,a904 int,a905 int,a906 int,a907 int,a908 int,a909 int,a910 int,a911 int,a912 int,a913 int,a914 int,a915 int,a916 int,a917 int,a918 int,a919 int,a920 int,a921 int,a922 int,a923 int,a924 int,a925 int,a926 int,a927 int,a928 int,a929 int,a930 int,a931 int,a932 int,a933 int,a934 int,a935 int,a936 int,a937 int,a938 int,a939 int,a940 int,a941 int,a942 int,a943 int,a944 int,a945 int,a946 int,a947 int,a948 int,a949 int,a950 int,a951 int,a952 int,a953 int,a954 int,a955 int,a956 int,a957 int,a958 int,a959 int,a960 int,a961 int,a962 int,a963 int,a964 int,a965 int,a966 int,a967 int,a968 int,a969 int,a970 int,a971 int,a972 int,a973 int,a974 int,a975 int,a976 int,a977 int,a978 int,a979 int,a980 int,a981 int,a982 int,a983 int,a984 int,a985 int,a986 int,a987 int,a988 int,a989 int,a990 int,a991 int,a992 int,a993 int,a994 int,a995 int,a996 int,a997 int,a998 int,a999 int,a1000 int,a1001 int,a1002 int,a1003 int,a1004 int,a1005 int,a1006 int,a1007 int,a1008 int,a1009 int,a1010 int,a1011 int,a1012 int,a1013 int,a1014 int,a1015 int); +alter table t_column_too_many add column a_512 int; +alter table t_column_too_many add column a_513 int; +Error 1117 (HY000): Too many columns +drop table if exists t; +CREATE TABLE t ( +c01 varchar(255) NOT NULL, +c02 varchar(255) NOT NULL, +c03 varchar(255) NOT NULL, +c04 varchar(255) DEFAULT NULL, +c05 varchar(255) DEFAULT NULL, +c06 varchar(255) DEFAULT NULL, +PRIMARY KEY (c01,c02,c03) clustered, +KEY c04 (c04) +); +drop table t; +CREATE TABLE t ( +c01 varchar(255) NOT NULL, +c02 varchar(255) NOT NULL, +c03 varchar(255) NOT NULL, +c04 varchar(255) NOT NULL, +c05 varchar(255) DEFAULT NULL, +c06 varchar(255) DEFAULT NULL, +PRIMARY KEY (c01,c02,c03,c04) clustered +); +Error 1071 (42000): Specified key was too long (4080 bytes); max key length is 3072 bytes +CREATE TABLE t ( +c01 varchar(255) NOT NULL, +c02 varchar(255) NOT NULL, +c03 varchar(255) NOT NULL, +c04 varchar(255) DEFAULT NULL, +c05 varchar(255) DEFAULT NULL, +c06 varchar(255) DEFAULT NULL, +PRIMARY KEY (c01,c02,c03) clustered, +unique key c04 (c04) +); +drop table t; +CREATE TABLE t ( +c01 varchar(255) NOT NULL, +c02 varchar(255) NOT NULL, +c03 varchar(255) NOT NULL, +c04 varchar(255) DEFAULT NULL, +c05 varchar(255) DEFAULT NULL, +c06 varchar(255) DEFAULT NULL, +PRIMARY KEY (c01,c02) clustered +); +create index idx1 on t(c03); +create index idx2 on t(c03, c04); +create unique index uk2 on t(c03, c04); +drop table t; +CREATE TABLE t ( +c01 varchar(255) NOT NULL, +c02 varchar(255) NOT NULL, +c03 varchar(255) NOT NULL, +c04 varchar(255) DEFAULT NULL, +c05 varchar(255) DEFAULT NULL, +c06 varchar(255) DEFAULT NULL, +Index idx1(c03), +PRIMARY KEY (c01,c02) clustered, +unique index uk1(c06) +); +alter table t change c03 c10 varchar(256) default null; +alter table t change c10 c100 varchar(1024) default null; +Error 1071 (42000): Specified key was too long (4096 bytes); max key length is 3072 bytes +alter table t modify c10 varchar(600) default null; +alter table t modify c06 varchar(600) default null; +alter table t modify c01 varchar(510); +drop table if exists t2; +create table t2 like t; +drop table if exists t; +create table t( +a int, +b varchar(100), +c int, +INDEX idx_c(c)) +PARTITION BY RANGE COLUMNS( a ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21) +); +insert into t values (4, 'xxx', 4); +insert into t values (4, 'xxx', 9); +insert into t values (17, 'xxx', 12); +alter table t add unique index idx_a(a); +Error 1062 (23000): Duplicate entry '4' for key 't.idx_a' +delete from t where a = 4; +alter table t add unique index idx_a(a); +alter table t add unique index idx_ac(a, c); +alter table t add unique index idx_b(b); +Error 1503 (HY000): A UNIQUE INDEX must include all columns in the table's partitioning function +drop table if exists t; +drop table if exists t_ft; +create table t_ft (a text, fulltext key (a)); +show warnings; +Level Code Message +Warning 1214 The used table type doesn't support FULLTEXT indexes +alter table t_ft add fulltext key (a); +show warnings; +Level Code Message +Warning 1214 The used table type doesn't support FULLTEXT indexes +show create table t_ft; +Table Create Table +t_ft CREATE TABLE `t_ft` ( + `a` text DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t_ft; +drop table if exists t; +create table t (a int default b'1'); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT '1' +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t; +drop table if exists t; +create table t (c int(10), c1 varchar(256) default (uuid())); +alter table t add column c2 varchar(256) default (uuid()); +Error 1674 (HY000): Statement is unsafe because it uses a system function that may return a different value on the slave +insert into t(c) values (1),(2),(3),(4),(5),(6),(7),(8),(9),(10); +select count(distinct c1) from t; +count(distinct c1) +10 +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `c` int(10) DEFAULT NULL, + `c1` varchar(256) DEFAULT uuid() +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table if exists t1; +create table t1 (a int(11) not null auto_increment key, b int(11), c bigint, unique key (a, b, c)); +alter table t1 drop index a; +drop table if exists t1; +create table t1 (a int auto_increment, unique key (a)); +alter table t1 drop index a; +drop table if exists t1; +create table t1 (a int(11) not null auto_increment, b int(11), c bigint, unique key (a, b, c)); +alter table t1 drop index a; +drop table if exists t1; +create table t1 (a int, b int as (-a) virtual, c int as (-a) stored); +insert into t1 values (1, default, default); +select * from t1; +a b c +1 -1 -1 +delete from t1; +insert into t1(a,b) values (1, default), (2, default); +select * from t1; +a b c +1 -1 -1 +2 -2 -2 +delete from t1; +insert into t1(b) values (default); +select * from t1; +a b c +NULL NULL NULL +delete from t1; +insert into t1(c) values (default); +select * from t1; +a b c +NULL NULL NULL +delete from t1; +drop table if exists t2; +create table t2 like t1; +alter table t2 add index idx1(a); +alter table t2 add index idx2(b); +insert into t2 values (1, default, default); +select * from t2; +a b c +1 -1 -1 +delete from t2; +alter table t2 drop index idx1; +alter table t2 drop index idx2; +insert into t2 values (1, default, default); +select * from t2; +a b c +1 -1 -1 +drop table if exists t3; +create table t3 (gc1 int as (r+1), gc2 int as (r+1) stored, gc3 int as (gc2+1), gc4 int as (gc1+1) stored, r int); +insert into t3 values (default, default, default, default, 1); +select * from t3; +gc1 gc2 gc3 gc4 r +2 2 3 3 1 +drop table if exists t4; +create table t4 (a int key, b int, c int as (a+1), d int as (b+1) stored); +insert into t4 values (1, 10, default, default); +select * from t4; +a b c d +1 10 2 11 +replace into t4 values (1, 20, default, default); +select * from t4; +a b c d +1 20 2 21 +drop table if exists t5; +create table t5 (a int default 10, b int as (a+1)); +insert into t5 values (20, default(a)); +Error 3105 (HY000): The value specified for generated column 'b' in table 't5' is not allowed. +drop table t1, t2, t3, t4, t5; +drop table if exists t; +create table t(a int, b json); +insert into t values (1, '{"a": 1}'); +alter table t add index idx((cast(b->'$.a' as char(255)))); +select * from t force index(idx); +a b +1 {"a": 1} +select * from t ignore index(idx); +a b +1 {"a": 1} +alter table t add index idx1((cast(b->>'$.a' as char(255)))); +select * from t force index(idx1); +a b +1 {"a": 1} +select * from t ignore index(idx1); +a b +1 {"a": 1} +alter table t add index idx2((json_type(b))); +select * from t force index(idx2) where json_type(b) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx2) where json_type(b) = 'OBJECT'; +a b +1 {"a": 1} +alter table t add index idx_wrong((b->'$.a')); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((b->>'$.a')); +Error 3757 (HY000): Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST +alter table t add index idx_wrong((json_pretty(b))); +Error 3757 (HY000): Cannot create an expression index on an expression that returns a BLOB or TEXT. Please consider using CAST +drop table if exists t; +create table t(a char(255), index idx((json_quote(a)))); +Error 1071 (42000): Specified key was too long (6128 bytes); max key length is 3072 bytes +create table t(a char(40)); +insert into t values ('[1, 2, 3]'); +alter table t add index idx3((json_quote(a))); +select * from t force index(idx3) where json_quote(a) = '"[1, 2, 3]"'; +a +[1, 2, 3] +select * from t ignore index(idx3) where json_quote(a) = '"[1, 2, 3]"'; +a +[1, 2, 3] +drop table if exists t; +create table t(a int, b json); +alter table t add index idx_wrong((json_array(b))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_object('key', b))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_merge_preserve(b, '{"k": "v"}'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_set(b, '$.a', 'v'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_insert(b, '$.a', 'v'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_replace(b, '$.a', 'v'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_remove(b, '$.a'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_array_append(b, '$.a', 1))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_merge_patch(b, '{"k": "v"}'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_search(b, 'one', 'a'))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +alter table t add index idx_wrong((json_keys(b))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +drop table if exists t; +create table t(a int, b json); +insert into t values (1, '{"a": 1}'); +alter table t add index idx0((json_type(json_search(b, 'one', 'a')))); +alter table t add index idx1((json_type(json_array(b)))); +alter table t add index idx2((json_type(json_object('key', b)))); +alter table t add index idx3((json_type(json_merge_preserve(b, '{"k": "v"}')))); +alter table t add index idx4((json_type(json_set(b, '$.a', 'v')))); +alter table t add index idx5((json_type(json_insert(b, '$.a', 'v')))); +alter table t add index idx6((json_type(json_replace(b, '$.a', 'v')))); +alter table t add index idx7((json_type(json_remove(b, '$.a')))); +alter table t add index idx8((json_type(json_array_append(b, '$.a', 1)))); +alter table t add index idx9((json_type(json_merge_patch(b, '{"k": "v"}')))); +alter table t add index idx10((json_type(json_keys(b)))); +alter table t add index idx11((cast(json_quote(cast(a as char(10))) as char(64)))); +alter table t add index idx12((json_storage_size(b))); +alter table t add index idx13((json_depth(b))); +alter table t add index idx14((json_length(b))); +select * from t force index(idx0) where json_type(json_search(b, 'one', 'a')) is NULL; +a b +1 {"a": 1} +select * from t force index(idx1) where json_type(json_array(b)) = 'ARRAY'; +a b +1 {"a": 1} +select * from t force index(idx2) where json_type(json_object('key', b)) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx3) where json_type(json_merge_preserve(b, '{"k": "v"}')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx4) where json_type(json_set(b, '$.a', 'v')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx5) where json_type(json_insert(b, '$.a', 'v')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx6) where json_type(json_replace(b, '$.a', 'v')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx7) where json_type(json_remove(b, '$.a')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx8) where json_type(json_array_append(b, '$.a', 1)) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx9) where json_type(json_merge_patch(b, '{"k": "v"}')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t force index(idx10) where json_type(json_keys(b)) = 'ARRAY'; +a b +1 {"a": 1} +select * from t force index(idx11) where cast(json_quote(cast(a as char(10))) as char(64)) = '"1"'; +a b +1 {"a": 1} +select * from t force index(idx12) where json_storage_size(b) > 1; +a b +1 {"a": 1} +select * from t force index(idx13) where json_depth(b) > 0; +a b +1 {"a": 1} +select * from t force index(idx14) where json_length(b) > 0; +a b +1 {"a": 1} +select * from t ignore index(idx0) where json_type(json_search(b, 'one', 'a')) is NULL; +a b +1 {"a": 1} +select * from t ignore index(idx1) where json_type(json_array(b)) = 'ARRAY'; +a b +1 {"a": 1} +select * from t ignore index(idx2) where json_type(json_object('key', b)) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx3) where json_type(json_merge_preserve(b, '{"k": "v"}')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx4) where json_type(json_set(b, '$.a', 'v')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx5) where json_type(json_insert(b, '$.a', 'v')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx6) where json_type(json_replace(b, '$.a', 'v')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx7) where json_type(json_remove(b, '$.a')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx8) where json_type(json_array_append(b, '$.a', 1)) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx9) where json_type(json_merge_patch(b, '{"k": "v"}')) = 'OBJECT'; +a b +1 {"a": 1} +select * from t ignore index(idx10) where json_type(json_keys(b)) = 'ARRAY'; +a b +1 {"a": 1} +select * from t ignore index(idx11) where cast(json_quote(cast(a as char(10))) as char(64)) = '"1"'; +a b +1 {"a": 1} +select * from t ignore index(idx12) where json_storage_size(b) > 1; +a b +1 {"a": 1} +select * from t ignore index(idx13) where json_depth(b) > 0; +a b +1 {"a": 1} +select * from t ignore index(idx14) where json_length(b) > 0; +a b +1 {"a": 1} +drop table if exists t; +create table t( +a int, +b varchar(100), +c int) +PARTITION BY RANGE ( a ) ( +PARTITION p0 VALUES LESS THAN (6), +PARTITION p1 VALUES LESS THAN (11), +PARTITION p2 VALUES LESS THAN (16), +PARTITION p3 VALUES LESS THAN (21) +); +insert into t values (1, 'test', 2), (12, 'test', 3), (15, 'test', 10), (20, 'test', 20); +alter table t add index idx((a+c)); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) DEFAULT NULL, + `b` varchar(100) DEFAULT NULL, + `c` int(11) DEFAULT NULL, + KEY `idx` ((`a` + `c`)) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +PARTITION BY RANGE (`a`) +(PARTITION `p0` VALUES LESS THAN (6), + PARTITION `p1` VALUES LESS THAN (11), + PARTITION `p2` VALUES LESS THAN (16), + PARTITION `p3` VALUES LESS THAN (21)) +select * from t order by a; +a b c +1 test 2 +12 test 3 +15 test 10 +20 test 20 +drop table if exists t; +drop table if exists t1; +create table t(a int auto_increment key clustered) auto_id_cache 100; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL AUTO_INCREMENT, + PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=100 */ +insert into t values(); +select * from t; +a +1 +delete from t; +rename table t to t1; +insert into t1 values(); +select * from t1; +a +2 +drop table if exists t; +drop table if exists t1; +create table t(a int) auto_id_cache 100; +insert into t values(); +select _tidb_rowid from t; +_tidb_rowid +1 +delete from t; +rename table t to t1; +insert into t1 values(); +select _tidb_rowid from t1; +_tidb_rowid +2 +drop table if exists t; +drop table if exists t1; +create table t(a int null, b int auto_increment unique) auto_id_cache 100; +insert into t(b) values(NULL); +select b, _tidb_rowid from t; +b _tidb_rowid +1 2 +delete from t; +rename table t to t1; +insert into t1(b) values(NULL); +select b, _tidb_rowid from t1; +b _tidb_rowid +3 4 +delete from t1; +alter table t1 auto_id_cache 200; +show create table t1; +Table Create Table +t1 CREATE TABLE `t1` ( + `a` int(11) DEFAULT NULL, + `b` int(11) NOT NULL AUTO_INCREMENT, + UNIQUE KEY `b` (`b`) +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin AUTO_INCREMENT=101 /*T![auto_id_cache] AUTO_ID_CACHE=200 */ +insert into t1(b) values(NULL); +select b, _tidb_rowid from t1; +b _tidb_rowid +101 102 +delete from t1; +rename table t1 to t; +insert into t(b) values(NULL); +select b, _tidb_rowid from t; +b _tidb_rowid +103 104 +delete from t; +drop table if exists t; +drop table if exists t1; +create table t(a int auto_increment key clustered) auto_id_cache 3; +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL AUTO_INCREMENT, + PRIMARY KEY (`a`) /*T![clustered_index] CLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin /*T![auto_id_cache] AUTO_ID_CACHE=3 */ +insert into t(a) values(NULL),(NULL),(NULL); +insert into t(a) values(NULL); +select a from t; +a +1 +2 +3 +4 +delete from t; +rename table t to t1; +insert into t1(a) values(NULL); +select a from t1; +a +5 +drop table if exists t; +create table t(a int) auto_id_cache = 9223372036854775808; +Error 1105 (HY000): table option auto_id_cache overflows int64 +create table t(a int) auto_id_cache = 9223372036854775807; +alter table t auto_id_cache = 9223372036854775808; +Error 1105 (HY000): table option auto_id_cache overflows int64 +drop table if exists t, t1, t2, t3; +create table t(a int NOT NULL, b int, key(a), unique(b) invisible); +select distinct index_name, is_visible from information_schema.statistics where table_schema = 'ddl__db_integration' and table_name = 't' order by index_name; +index_name is_visible +a YES +b NO +alter table t alter index a invisible; +select distinct index_name, is_visible from information_schema.statistics where table_schema = 'ddl__db_integration' and table_name = 't' order by index_name; +index_name is_visible +a NO +b NO +alter table t alter index b visible; +select distinct index_name, is_visible from information_schema.statistics where table_schema = 'ddl__db_integration' and table_name = 't' order by index_name; +index_name is_visible +a NO +b YES +alter table t alter index b invisible; +select distinct index_name, is_visible from information_schema.statistics where table_schema = 'ddl__db_integration' and table_name = 't' order by index_name; +index_name is_visible +a NO +b NO +alter table t alter index non_exists_idx visible; +Error 1176 (42000): Key 'non_exists_idx' doesn't exist in table 't' +create table t1(a int NOT NULL, unique(a)); +alter table t1 alter index a invisible; +Error 3522 (HY000): A primary key index cannot be invisible +create table t2(a int, primary key(a)); +alter table t2 alter index PRIMARY invisible; +Error 1064 (42000): You have an error in your SQL syntax; check the manual that corresponds to your TiDB version for the right syntax to use line 1 column 34 near "PRIMARY invisible;" +create table t3(a int NOT NULL, b int); +alter table t3 add index idx((a+b)); +select distinct index_name, is_visible from information_schema.statistics where table_schema = 'ddl__db_integration' and table_name = 't3' order by index_name; +index_name is_visible +idx YES +alter table t3 alter index idx invisible; +select distinct index_name, is_visible from information_schema.statistics where table_schema = 'ddl__db_integration' and table_name = 't3' order by index_name; +index_name is_visible +idx NO +drop table if exists t_drop_last_column, t_drop_last_columns; +create table t_drop_last_column(x int, key((1+1))); +alter table t_drop_last_column drop column x; +Error 1113 (42000): A table must have at least 1 column +create table t_drop_last_columns(x int, y int, key((1+1))); +alter table t_drop_last_columns drop column x, drop column y; +Error 1113 (42000): A table must have at least 1 column +drop table if exists t_drop_last_column, t_drop_last_columns; +drop table if exists issue20741_2; +create table issue20741_2(id int primary key, c int); +insert into issue20741_2(id, c) values(1, 2), (2, 2); +alter table issue20741_2 add column cc set('a', 'b', 'c', 'd') not null; +update issue20741_2 set c=2 where id=1; +select * from issue20741_2; +id c cc +1 2 +2 2 +select * from issue20741_2 where cc = 0; +id c cc +1 2 +2 2 +select * from issue20741_2 where cc = 1; +id c cc +insert into issue20741_2(id, c) values (3, 3); +Error 1364 (HY000): Field 'cc' doesn't have a default value +drop table if exists t; +create table t( col decimal(1,2) not null default 0); +Error 1427 (42000): For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column 'col'). +drop table if exists tplacement1, tplacement2; +drop database if exists db2; +drop placement policy if exists x; +create placement policy x primary_region='r1' regions='r1'; +create temporary table tplacement2 (id int) placement policy='x'; +Error 8006 (HY000): `PLACEMENT` is unsupported on temporary tables. +create global temporary table tplacement1 (id int) on commit delete rows; +alter table tplacement1 placement policy='x'; +Error 8006 (HY000): `placement` is unsupported on temporary tables. +create temporary table tplacement2 (id int); +alter table tplacement2 placement policy='x'; +Error 8200 (HY000): TiDB doesn't support ALTER TABLE for local temporary table +create database db2 placement policy x; +create global temporary table db2.tplacement3 (id int) on commit delete rows; +show create table db2.tplacement3; +Table Create Table +tplacement3 CREATE GLOBAL TEMPORARY TABLE `tplacement3` ( + `id` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS +create temporary table db2.tplacement4 (id int); +show create table db2.tplacement4; +Table Create Table +tplacement4 CREATE TEMPORARY TABLE `tplacement4` ( + `id` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +create table db2.t1 (a int) placement policy 'default'; +create global temporary table db2.tplacement5 like db2.t1 on commit delete rows; +show create table db2.tplacement5; +Table Create Table +tplacement5 CREATE GLOBAL TEMPORARY TABLE `tplacement5` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin ON COMMIT DELETE ROWS +create temporary table db2.tplacement6 like db2.t1; +show create table db2.tplacement6; +Table Create Table +tplacement6 CREATE TEMPORARY TABLE `tplacement6` ( + `a` int(11) DEFAULT NULL +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin +drop table db2.tplacement6; +drop table db2.tplacement5; +drop table db2.t1; +drop table db2.tplacement3; +drop database db2; +drop table tplacement1, tplacement2; +drop placement policy x; +drop table if exists tb, tb2, temp, temp1, ltemp1, ltemp2; +create table tb(id int); +create table tb2(id int); +create global temporary table temp(id int) on commit delete rows; +create global temporary table temp1(id int) on commit delete rows; +create temporary table ltemp1(id int); +create temporary table ltemp2(id int); +drop global temporary table tb; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table ddl__db_integration.tb; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table ltemp1; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table ddl__db_integration.ltemp1; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table ltemp1, temp; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table temp, ltemp1; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table xxx, ltemp1; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table xxx; +Error 1051 (42S02): Unknown table 'ddl__db_integration.xxx' +drop global temporary table if exists tb; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table if exists ltemp1; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table if exists xxx; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.xxx' +drop global temporary table if exists xxx,tb; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table if exists ddl__db_integration.tb; +Error 8007 (HY000): `drop global temporary table` can only drop global temporary table +drop global temporary table temp; +select * from temp; +Error 1146 (42S02): Table 'ddl__db_integration.temp' doesn't exist +drop global temporary table ddl__db_integration.temp1; +select * from temp2; +Error 1146 (42S02): Table 'ddl__db_integration.temp2' doesn't exist +create global temporary table temp (id int) on commit delete rows; +create global temporary table temp1 (id int) on commit delete rows; +drop global temporary table temp, temp1; +select * from temp; +Error 1146 (42S02): Table 'ddl__db_integration.temp' doesn't exist +select * from temp1; +Error 1146 (42S02): Table 'ddl__db_integration.temp1' doesn't exist +create global temporary table temp (id int) on commit delete rows; +create global temporary table temp1 (id int) on commit delete rows; +drop global temporary table if exists temp; +show warnings; +Level Code Message +select * from temp; +Error 1146 (42S02): Table 'ddl__db_integration.temp' doesn't exist +drop table if exists tb, tb2, temp, temp1, ltemp1, ltemp2; +drop table if exists tb, tb2, temp, temp1, ltemp1, ltemp2, testt.ltemp3; +create table tb(id int); +create table tb2(id int); +insert into tb2 values(1); +create temporary table tb2(id int); +create global temporary table temp(id int) on commit delete rows; +create global temporary table temp1(id int) on commit delete rows; +create temporary table ltemp1(id int); +create temporary table ltemp2(id int); +create database if not exists testt; +create temporary table testt.ltemp3(id int); +drop temporary table tb; +Error 1051 (42S02): Unknown table 'ddl__db_integration.tb' +drop temporary table ddl__db_integration.tb; +Error 1051 (42S02): Unknown table 'ddl__db_integration.tb' +drop temporary table temp1; +Error 1051 (42S02): Unknown table 'ddl__db_integration.temp1' +drop temporary table ddl__db_integration.temp1; +Error 1051 (42S02): Unknown table 'ddl__db_integration.temp1' +drop temporary table ltemp1, tb; +Error 1051 (42S02): Unknown table 'ddl__db_integration.tb' +drop temporary table temp, ltemp1; +Error 1051 (42S02): Unknown table 'ddl__db_integration.temp' +drop temporary table xxx, ltemp1; +Error 1051 (42S02): Unknown table 'ddl__db_integration.xxx' +drop temporary table xxx; +Error 1051 (42S02): Unknown table 'ddl__db_integration.xxx' +drop temporary table if exists xxx; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.xxx' +drop temporary table if exists ltemp1, xxx; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.xxx' +drop temporary table if exists tb1, xxx; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.tb1,ddl__db_integration.xxx' +drop temporary table if exists temp1; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.temp1' +drop temporary table if exists temp1, xxx; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.temp1,ddl__db_integration.xxx' +drop temporary table if exists testt.ltemp4; +show warnings; +Level Code Message +Note 1051 Unknown table 'testt.ltemp4' +drop temporary table if exists testt.ltemp3, tb1; +show warnings; +Level Code Message +Note 1051 Unknown table 'ddl__db_integration.tb1' +drop temporary table ltemp1; +select * from ltemp1; +Error 1146 (42S02): Table 'ddl__db_integration.ltemp1' doesn't exist +drop temporary table ddl__db_integration.ltemp2; +select * from ltemp2; +Error 1146 (42S02): Table 'ddl__db_integration.ltemp2' doesn't exist +drop temporary table tb2; +select * from tb2; +id +1 +create temporary table ltemp1 (id int); +create temporary table ltemp2 (id int); +drop temporary table testt.ltemp3, ltemp1; +select * from testt.ltemp3; +Error 1146 (42S02): Table 'testt.ltemp3' doesn't exist +select * from ltemp1; +Error 1146 (42S02): Table 'ddl__db_integration.ltemp1' doesn't exist +drop temporary table if exists ltemp2; +show warnings; +Level Code Message +select * from ltemp2; +Error 1146 (42S02): Table 'ddl__db_integration.ltemp2' doesn't exist +drop table if exists tb, tb2, temp, temp1, ltemp1, ltemp2, testt.ltemp3; +drop database testt; +drop table if exists t1; +create table t1 (id int); +insert into t1 values(1); +drop table if exists t2; +create table t2 (id int); +insert into t2 values(1); +drop view if exists v1; +create view v1 as select 1,1; +select * from v1; +1 Name_exp_1 +1 1 +drop view if exists v1; +create view v1 as select 1, 2, 1, 2, 1, 2, 1, 2; +select * from v1; +1 2 Name_exp_1 Name_exp_2 Name_exp_1_1 Name_exp_1_2 Name_exp_2_1 Name_exp_2_2 +1 2 1 2 1 2 1 2 +drop view if exists v1; +create view v1 as select 't', 't', 1 as t; +select * from v1; +Name_exp_t Name_exp_1_t t +t t 1 +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select 1, 1 union all select 1, 1; +show create view v1; +View Create View character_set_client collation_connection +v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`1`, `Name_exp_1`) AS SELECT 1 AS `1`,1 AS `Name_exp_1` UNION ALL SELECT 1 AS `1`,1 AS `1` utf8mb4 utf8mb4_general_ci +select * from v1; +1 Name_exp_1 +1 1 +1 1 +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select 'id', id from t1; +show create view v1; +View Create View character_set_client collation_connection +v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`Name_exp_id`, `id`) AS SELECT _UTF8MB4'id' AS `Name_exp_id`,`id` AS `id` FROM `ddl__db_integration`.`t1` utf8mb4 utf8mb4_general_ci +select * from v1; +Name_exp_id id +id 1 +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select 1, (select id from t1 where t1.id=t2.id) as '1' from t2; +show create view v1; +View Create View character_set_client collation_connection +v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`Name_exp_1`, `1`) AS SELECT 1 AS `Name_exp_1`,(SELECT `id` AS `id` FROM `ddl__db_integration`.`t1` WHERE `t1`.`id`=`t2`.`id`) AS `1` FROM `ddl__db_integration`.`t2` utf8mb4 utf8mb4_general_ci +select * from v1; +Name_exp_1 1 +1 1 +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select 1 as 'abs(t1.id)', abs(t1.id) from t1; +show create view v1; +View Create View character_set_client collation_connection +v1 CREATE ALGORITHM=UNDEFINED DEFINER=`root`@`127.0.0.1` SQL SECURITY DEFINER VIEW `v1` (`abs(t1.id)`, `Name_exp_abs(t1.id)`) AS SELECT 1 AS `abs(t1.id)`,ABS(`t1`.`id`) AS `Name_exp_abs(t1.id)` FROM `ddl__db_integration`.`t1` utf8mb4 utf8mb4_general_ci +select * from v1; +abs(t1.id) Name_exp_abs(t1.id) +1 1 +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select 1 as t,1 as t; +Error 1060 (42S21): Duplicate column name 't' +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select 1 as id, id from t1; +Error 1060 (42S21): Duplicate column name 'id' +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select * from t1 left join t2 on t1.id=t2.id; +Error 1060 (42S21): Duplicate column name 'id' +drop view if exists v1; +create definer=`root`@`127.0.0.1` view v1 as select t1.id, t2.id from t1,t2 where t1.id=t2.id; +Error 1060 (42S21): Duplicate column name 'id' +drop view if exists v1; +drop table t2; +drop table t1; +drop table if exists t; +create table t(a int) partition by range (a) (partition p0 values less than (0), partition `p1 ` values less than (3)); +Error 1567 (HY000): Incorrect partition name +create table t(a int) partition by range (a) (partition `` values less than (0), partition `p1` values less than (3)); +Error 1567 (HY000): Incorrect partition name +create table t(a int) partition by range (a) (partition `p0` values less than (0), partition `p1` values less than (3)); +alter table t add partition (partition `p2 ` values less than (5)); +Error 1567 (HY000): Incorrect partition name +drop table if exists reg_like; +create table reg_like(a varchar(50), b varchar(50), c int generated always as (regexp_like(a, b))); +insert into reg_like(a, b) values('123', '2'); +insert into reg_like(a, b) values('456', '1'); +select * from reg_like; +a b c +123 2 1 +456 1 0 +drop table if exists reg_sub; +create table reg_sub(a varchar(50),b varchar(50),c varchar(50) generated always as (regexp_substr(a, b))); +insert into reg_sub(a, b) values('abcd', 'bc.'); +insert into reg_sub(a, b) values('1234', '23.'); +select * from reg_sub; +a b c +abcd bc. bcd +1234 23. 234 +drop table if exists reg_instr; +create table reg_instr(a varchar(50),b varchar(50),c varchar(50) generated always as (regexp_instr(a, b))); +insert into reg_instr(a, b) values('abcd', 'bc.'); +insert into reg_instr(a, b) values('1234', '23.'); +select * from reg_instr; +a b c +abcd bc. 2 +1234 23. 2 +drop table if exists reg_replace; +create table reg_replace(a varchar(50),b varchar(50),c varchar(50),d varchar(50) generated always as (regexp_replace(a, b, c))); +insert into reg_replace(a, b, c) values('abcd', 'bc.', 'xzx'); +insert into reg_replace(a, b, c) values('1234', '23.', 'xzx'); +select * from reg_replace; +a b c d +abcd bc. xzx axzx +1234 23. xzx 1xzx +drop table if exists reg_like; +drop table if exists t; +CREATE TABLE t (id int, d varchar(255)) partition by range (id) (partition p0 values less than (1000000), partition p1 values less than (2000000), partition p2 values less than (3000000)); +ALTER TABLE t REORGANIZE PARTITION p0,p2 INTO (PARTITION p0 VALUES LESS THAN (1000000)); +Error 8200 (HY000): Unsupported REORGANIZE PARTITION of RANGE; not adjacent partitions +ALTER TABLE t REORGANIZE PARTITION p0,p2 INTO (PARTITION p0 VALUES LESS THAN (4000000)); +Error 8200 (HY000): Unsupported REORGANIZE PARTITION of RANGE; not adjacent partitions +drop table if exists members, member_level; +CREATE TABLE members ( +id int, +fname varchar(255), +lname varchar(255), +dob date, +data json +) +PARTITION BY RANGE (YEAR(dob)) ( +PARTITION pBefore1950 VALUES LESS THAN (1950), +PARTITION p1950 VALUES LESS THAN (1960), +PARTITION p1960 VALUES LESS THAN (1970), +PARTITION p1970 VALUES LESS THAN (1980), +PARTITION p1980 VALUES LESS THAN (1990), +PARTITION p1990 VALUES LESS THAN (2000)); +CREATE TABLE member_level ( +id int, +level int, +achievements json +) +PARTITION BY LIST (level) ( +PARTITION l1 VALUES IN (1), +PARTITION l2 VALUES IN (2), +PARTITION l3 VALUES IN (3), +PARTITION l4 VALUES IN (4), +PARTITION l5 VALUES IN (5)); +ALTER TABLE members DROP PARTITION p1990; +ALTER TABLE member_level DROP PARTITION l5; +ALTER TABLE members TRUNCATE PARTITION p1980; +ALTER TABLE member_level TRUNCATE PARTITION l4; +ALTER TABLE members ADD PARTITION (PARTITION `p1990to2010` VALUES LESS THAN (2010)); +ALTER TABLE member_level ADD PARTITION (PARTITION l5_6 VALUES IN (5,6)); +ALTER TABLE members ADD PARTITION (PARTITION p1990 VALUES LESS THAN (2000)); +Error 1493 (HY000): VALUES LESS THAN value must be strictly increasing for each partition +ALTER TABLE members REORGANIZE PARTITION p1990to2010 INTO +(PARTITION p1990 VALUES LESS THAN (2000), +PARTITION p2000 VALUES LESS THAN (2010), +PARTITION p2010 VALUES LESS THAN (2020), +PARTITION p2020 VALUES LESS THAN (2030), +PARTITION pMax VALUES LESS THAN (MAXVALUE)); +ALTER TABLE member_level REORGANIZE PARTITION l5_6 INTO +(PARTITION l5 VALUES IN (5), +PARTITION l6 VALUES IN (6)); +ALTER TABLE members REORGANIZE PARTITION pBefore1950,p1950 INTO (PARTITION pBefore1960 VALUES LESS THAN (1960)); +ALTER TABLE member_level REORGANIZE PARTITION l1,l2 INTO (PARTITION l1_2 VALUES IN (1,2)); +ALTER TABLE members REORGANIZE PARTITION pBefore1960,p1960,p1970,p1980,p1990,p2000,p2010,p2020,pMax INTO +(PARTITION p1800 VALUES LESS THAN (1900), +PARTITION p1900 VALUES LESS THAN (2000), +PARTITION p2000 VALUES LESS THAN (2100)); +ALTER TABLE member_level REORGANIZE PARTITION l1_2,l3,l4,l5,l6 INTO +(PARTITION lOdd VALUES IN (1,3,5), +PARTITION lEven VALUES IN (2,4,6)); +ALTER TABLE members REORGANIZE PARTITION p1800,p2000 INTO (PARTITION p2000 VALUES LESS THAN (2100)); +Error 8200 (HY000): Unsupported REORGANIZE PARTITION of RANGE; not adjacent partitions +INSERT INTO members VALUES (313, "John", "Doe", "2022-11-22", NULL); +ALTER TABLE members REORGANIZE PARTITION p2000 INTO (PARTITION p2000 VALUES LESS THAN (2050)); +ALTER TABLE members REORGANIZE PARTITION p2000 INTO (PARTITION p2000 VALUES LESS THAN (2020)); +Error 1526 (HY000): Table has no partition for value 2022 +INSERT INTO member_level (id, level) values (313, 6); +ALTER TABLE member_level REORGANIZE PARTITION lEven INTO (PARTITION lEven VALUES IN (2,4)); +Error 1526 (HY000): Table has no partition for value 6 +select @@global.tidb_enable_ddl; +@@global.tidb_enable_ddl +1 +set @@global.tidb_enable_ddl=false; +Error 8246 (HY000): Error happened when disabling DDL: can not disable ddl owner when it is the only one tidb instance +set @@global.tidb_enable_ddl=false; +Error 8246 (HY000): Error happened when disabling DDL: can not disable ddl owner when it is the only one tidb instance +select @@global.tidb_enable_ddl; +@@global.tidb_enable_ddl +1 +drop table if exists t; +create table t (id bigint, b varchar(20), index idxb(b)) partition by range(id) (partition p0 values less than (20), partition p1 values less than (100)); +alter table t reorganize partition p0 into (partition p01 values less than (10), partition p02 values less than (20)); +show warnings; +Level Code Message +Warning 1105 The statistics of related partitions will be outdated after reorganizing partitions. Please use 'ANALYZE TABLE' statement if you want to update it now +drop table if exists t; +create table t (a int, b real); +alter table t add primary key ((a+b)) nonclustered; +Error 3756 (HY000): The primary key cannot be an expression index +create table t(a int, index((cast(a as JSON)))); +Error 3753 (HY000): Cannot create an expression index on a function that returns a JSON or GEOMETRY value +drop table if exists t; +create table t (a int, b real); +alter table t add primary key ((a+b)) nonclustered; +Error 3756 (HY000): The primary key cannot be an expression index +alter table t add index ((rand())); +Error 3758 (HY000): Expression of expression index 'expression_index' contains a disallowed function +alter table t add index ((now()+1)); +Error 3758 (HY000): Expression of expression index 'expression_index' contains a disallowed function +alter table t add column (_V$_idx_0 int); +alter table t add index idx((a+1)); +Error 1060 (42S21): Duplicate column name '_V$_idx_0' +alter table t drop column _V$_idx_0; +alter table t add index idx((a+1)); +alter table t add column (_V$_idx_0 int); +Error 1060 (42S21): Duplicate column name '_V$_idx_0' +alter table t drop index idx; +alter table t add column (_V$_idx_0 int); +alter table t add column (_V$_expression_index_0 int); +alter table t add index ((a+1)); +Error 1060 (42S21): Duplicate column name '_V$_expression_index_0' +alter table t drop column _V$_expression_index_0; +alter table t add index ((a+1)); +alter table t drop column _V$_expression_index_0; +Error 1091 (42000): Can't DROP '_V$_expression_index_0'; check that column/key exists +alter table t add column e int as (_V$_expression_index_0 + 1); +Error 1054 (42S22): Unknown column '_v$_expression_index_0' in 'generated column function' +drop table if exists t; +create table t (j json, key k (((j,j)))); +Error 3800 (HY000): Expression of expression index 'k' cannot refer to a row value +create table t (j json, key k ((j+1),(j+1))); +create table t1 (col1 int, index ((concat('')))); +Error 3761 (HY000): The used storage engine cannot index the expression 'concat(_utf8mb4'')' +CREATE TABLE t1 (col1 INT, PRIMARY KEY ((ABS(col1))) NONCLUSTERED); +Error 3756 (HY000): The primary key cannot be an expression index +drop table if exists t; +create table t(id char(10) primary key, short_name char(10), name char(10), key n((upper(`name`)))); +update t t1 set t1.short_name='a' where t1.id='1'; +set @@tidb_enable_strict_double_type_check = 'ON'; +drop table if exists double_type_check; +create table double_type_check(id int, c double(10)); +Error 1149 (42000): You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use +set @@tidb_enable_strict_double_type_check = 'OFF'; +create table double_type_check(id int, c double(10)); +set @@tidb_enable_strict_double_type_check = default; +drop table if exists t1, t2, tt; +begin; +create temporary table t1(id int primary key, v int); +select * from t1; +id v +insert into t1 values(123, 456); +select * from t1 where id=123; +id v +123 456 +truncate table t1; +select * from t1 where id=123; +id v +commit; +create table tt(id int); +begin; +create temporary table t1(id int); +insert into tt select * from t1; +drop table tt; +create table t2(id int primary key, v int); +insert into t2 values(234, 567); +begin; +create temporary table t2(id int primary key, v int); +select * from t2 where id=234; +id v +commit; diff --git a/tests/integrationtest/r/executor/autoid.result b/tests/integrationtest/r/executor/autoid.result new file mode 100644 index 0000000000000..e27f47445cde5 --- /dev/null +++ b/tests/integrationtest/r/executor/autoid.result @@ -0,0 +1,739 @@ +drop table if exists t0, t1, t2, t3, t4, t5, t6, t7, t8, t9; +create table t0 (a tinyint signed key auto_increment) ; +insert into t0 values (-128); +insert into t0 values (); +insert into t0 values (0); +insert into t0 values (); +insert into t0 values (127); +insert into t0 values (); +Error 1690 (22003): constant 128 overflows tinyint +select * from t0 order by a; +a +-128 +1 +2 +3 +127 +drop table t0; +create table t1 (a tinyint unsigned key auto_increment) ; +insert into t1 values (0); +insert into t1 values (); +insert into t1 values (127); +insert into t1 values (); +insert into t1 values (255); +insert into t1 values (); +Error 1690 (22003): constant 256 overflows tinyint +select * from t1 order by a; +a +1 +2 +127 +128 +255 +drop table t1; +create table t2 (a smallint signed key auto_increment) ; +insert into t2 values (-32768); +insert into t2 values (); +insert into t2 values (0); +insert into t2 values (); +insert into t2 values (32767); +insert into t2 values (); +Error 1690 (22003): constant 32768 overflows smallint +select * from t2 order by a; +a +-32768 +1 +2 +3 +32767 +drop table t2; +create table t3 (a smallint unsigned key auto_increment) ; +insert into t3 values (0); +insert into t3 values (); +insert into t3 values (32767); +insert into t3 values (); +insert into t3 values (65535); +insert into t3 values (); +Error 1690 (22003): constant 65536 overflows smallint +select * from t3 order by a; +a +1 +2 +32767 +32768 +65535 +drop table t3; +create table t4 (a mediumint signed key auto_increment) ; +insert into t4 values (-8388608); +insert into t4 values (); +insert into t4 values (0); +insert into t4 values (); +insert into t4 values (8388607); +insert into t4 values (); +Error 1690 (22003): constant 8388608 overflows mediumint +select * from t4 order by a; +a +-8388608 +1 +2 +3 +8388607 +drop table t4; +create table t5 (a mediumint unsigned key auto_increment) ; +insert into t5 values (0); +insert into t5 values (); +insert into t5 values (8388607); +insert into t5 values (); +insert into t5 values (16777215); +insert into t5 values (); +Error 1690 (22003): constant 16777216 overflows mediumint +select * from t5 order by a; +a +1 +2 +8388607 +8388608 +16777215 +drop table t5; +create table t6 (a integer signed key auto_increment) ; +insert into t6 values (-2147483648); +insert into t6 values (); +insert into t6 values (0); +insert into t6 values (); +insert into t6 values (2147483647); +insert into t6 values (); +Error 1690 (22003): constant 2147483648 overflows int +select * from t6 order by a; +a +-2147483648 +1 +2 +3 +2147483647 +drop table t6; +create table t7 (a integer unsigned key auto_increment) ; +insert into t7 values (0); +insert into t7 values (); +insert into t7 values (2147483647); +insert into t7 values (); +insert into t7 values (4294967295); +insert into t7 values (); +Error 1690 (22003): constant 4294967296 overflows int +select * from t7 order by a; +a +1 +2 +2147483647 +2147483648 +4294967295 +drop table t7; +create table t8 (a bigint signed key auto_increment) ; +insert into t8 values (-9223372036854775808); +insert into t8 values (); +insert into t8 values (0); +insert into t8 values (); +insert into t8 values (9223372036854775807); +insert into t8 values (); +Error 1467 (HY000): Failed to read auto-increment value from storage engine +select * from t8 order by a; +a +-9223372036854775808 +1 +2 +3 +9223372036854775807 +drop table t8; +create table t9 (a bigint unsigned key auto_increment) ; +insert into t9 values (0); +insert into t9 values (); +insert into t9 values (9223372036854775807); +insert into t9 values (); +select * from t9 order by a; +a +1 +2 +9223372036854775807 +9223372036854775808 +drop table t9; +create table t0 (a tinyint signed key auto_increment) auto_id_cache 1; +insert into t0 values (-128); +insert into t0 values (); +insert into t0 values (0); +insert into t0 values (); +insert into t0 values (127); +insert into t0 values (); +Error 1690 (22003): constant 128 overflows tinyint +select * from t0 order by a; +a +-128 +1 +2 +3 +127 +drop table t0; +create table t1 (a tinyint unsigned key auto_increment) auto_id_cache 1; +insert into t1 values (0); +insert into t1 values (); +insert into t1 values (127); +insert into t1 values (); +insert into t1 values (255); +insert into t1 values (); +Error 1690 (22003): constant 256 overflows tinyint +select * from t1 order by a; +a +1 +2 +127 +128 +255 +drop table t1; +create table t2 (a smallint signed key auto_increment) auto_id_cache 1; +insert into t2 values (-32768); +insert into t2 values (); +insert into t2 values (0); +insert into t2 values (); +insert into t2 values (32767); +insert into t2 values (); +Error 1690 (22003): constant 32768 overflows smallint +select * from t2 order by a; +a +-32768 +1 +2 +3 +32767 +drop table t2; +create table t3 (a smallint unsigned key auto_increment) auto_id_cache 1; +insert into t3 values (0); +insert into t3 values (); +insert into t3 values (32767); +insert into t3 values (); +insert into t3 values (65535); +insert into t3 values (); +Error 1690 (22003): constant 65536 overflows smallint +select * from t3 order by a; +a +1 +2 +32767 +32768 +65535 +drop table t3; +create table t4 (a mediumint signed key auto_increment) auto_id_cache 1; +insert into t4 values (-8388608); +insert into t4 values (); +insert into t4 values (0); +insert into t4 values (); +insert into t4 values (8388607); +insert into t4 values (); +Error 1690 (22003): constant 8388608 overflows mediumint +select * from t4 order by a; +a +-8388608 +1 +2 +3 +8388607 +drop table t4; +create table t5 (a mediumint unsigned key auto_increment) auto_id_cache 1; +insert into t5 values (0); +insert into t5 values (); +insert into t5 values (8388607); +insert into t5 values (); +insert into t5 values (16777215); +insert into t5 values (); +Error 1690 (22003): constant 16777216 overflows mediumint +select * from t5 order by a; +a +1 +2 +8388607 +8388608 +16777215 +drop table t5; +create table t6 (a integer signed key auto_increment) auto_id_cache 1; +insert into t6 values (-2147483648); +insert into t6 values (); +insert into t6 values (0); +insert into t6 values (); +insert into t6 values (2147483647); +insert into t6 values (); +Error 1690 (22003): constant 2147483648 overflows int +select * from t6 order by a; +a +-2147483648 +1 +2 +3 +2147483647 +drop table t6; +create table t7 (a integer unsigned key auto_increment) auto_id_cache 1; +insert into t7 values (0); +insert into t7 values (); +insert into t7 values (2147483647); +insert into t7 values (); +insert into t7 values (4294967295); +insert into t7 values (); +Error 1690 (22003): constant 4294967296 overflows int +select * from t7 order by a; +a +1 +2 +2147483647 +2147483648 +4294967295 +drop table t7; +create table t8 (a bigint signed key auto_increment) auto_id_cache 1; +insert into t8 values (-9223372036854775808); +insert into t8 values (); +insert into t8 values (0); +insert into t8 values (); +insert into t8 values (9223372036854775807); +insert into t8 values (); +Error 1105 (HY000): auto increment action failed +select * from t8 order by a; +a +-9223372036854775808 +1 +2 +3 +9223372036854775807 +drop table t8; +create table t9 (a bigint unsigned key auto_increment) auto_id_cache 1; +insert into t9 values (0); +insert into t9 values (); +insert into t9 values (9223372036854775807); +insert into t9 values (); +select * from t9 order by a; +a +1 +2 +9223372036854775807 +9223372036854775808 +drop table t9; +create table t0 (a tinyint signed key auto_increment) auto_id_cache 100; +insert into t0 values (-128); +insert into t0 values (); +insert into t0 values (0); +insert into t0 values (); +insert into t0 values (127); +insert into t0 values (); +Error 1690 (22003): constant 128 overflows tinyint +select * from t0 order by a; +a +-128 +1 +2 +3 +127 +drop table t0; +create table t1 (a tinyint unsigned key auto_increment) auto_id_cache 100; +insert into t1 values (0); +insert into t1 values (); +insert into t1 values (127); +insert into t1 values (); +insert into t1 values (255); +insert into t1 values (); +Error 1690 (22003): constant 256 overflows tinyint +select * from t1 order by a; +a +1 +2 +127 +128 +255 +drop table t1; +create table t2 (a smallint signed key auto_increment) auto_id_cache 100; +insert into t2 values (-32768); +insert into t2 values (); +insert into t2 values (0); +insert into t2 values (); +insert into t2 values (32767); +insert into t2 values (); +Error 1690 (22003): constant 32768 overflows smallint +select * from t2 order by a; +a +-32768 +1 +2 +3 +32767 +drop table t2; +create table t3 (a smallint unsigned key auto_increment) auto_id_cache 100; +insert into t3 values (0); +insert into t3 values (); +insert into t3 values (32767); +insert into t3 values (); +insert into t3 values (65535); +insert into t3 values (); +Error 1690 (22003): constant 65536 overflows smallint +select * from t3 order by a; +a +1 +2 +32767 +32768 +65535 +drop table t3; +create table t4 (a mediumint signed key auto_increment) auto_id_cache 100; +insert into t4 values (-8388608); +insert into t4 values (); +insert into t4 values (0); +insert into t4 values (); +insert into t4 values (8388607); +insert into t4 values (); +Error 1690 (22003): constant 8388608 overflows mediumint +select * from t4 order by a; +a +-8388608 +1 +2 +3 +8388607 +drop table t4; +create table t5 (a mediumint unsigned key auto_increment) auto_id_cache 100; +insert into t5 values (0); +insert into t5 values (); +insert into t5 values (8388607); +insert into t5 values (); +insert into t5 values (16777215); +insert into t5 values (); +Error 1690 (22003): constant 16777216 overflows mediumint +select * from t5 order by a; +a +1 +2 +8388607 +8388608 +16777215 +drop table t5; +create table t6 (a integer signed key auto_increment) auto_id_cache 100; +insert into t6 values (-2147483648); +insert into t6 values (); +insert into t6 values (0); +insert into t6 values (); +insert into t6 values (2147483647); +insert into t6 values (); +Error 1690 (22003): constant 2147483648 overflows int +select * from t6 order by a; +a +-2147483648 +1 +2 +3 +2147483647 +drop table t6; +create table t7 (a integer unsigned key auto_increment) auto_id_cache 100; +insert into t7 values (0); +insert into t7 values (); +insert into t7 values (2147483647); +insert into t7 values (); +insert into t7 values (4294967295); +insert into t7 values (); +Error 1690 (22003): constant 4294967296 overflows int +select * from t7 order by a; +a +1 +2 +2147483647 +2147483648 +4294967295 +drop table t7; +create table t8 (a bigint signed key auto_increment) auto_id_cache 100; +insert into t8 values (-9223372036854775808); +insert into t8 values (); +insert into t8 values (0); +insert into t8 values (); +insert into t8 values (9223372036854775807); +insert into t8 values (); +Error 1467 (HY000): Failed to read auto-increment value from storage engine +select * from t8 order by a; +a +-9223372036854775808 +1 +2 +3 +9223372036854775807 +drop table t8; +create table t9 (a bigint unsigned key auto_increment) auto_id_cache 100; +insert into t9 values (0); +insert into t9 values (); +insert into t9 values (9223372036854775807); +insert into t9 values (); +select * from t9 order by a; +a +1 +2 +9223372036854775807 +9223372036854775808 +drop table t9; +create table t10 (a integer key auto_increment) auto_id_cache 1; +insert into t10 values (2147483648); +Error 1264 (22003): Out of range value for column 'a' at row 1 +insert into t10 values (-2147483649); +Error 1264 (22003): Out of range value for column 'a' at row 1 +drop table if exists t1, t2, t3, t11, t22, t33; +create table t1 (id int key auto_increment); +insert into t1 values (); +rename table t1 to t11; +insert into t11 values (); +select * from t11; +id +1 +2 +create table t2 (id int key auto_increment) auto_id_cache 1; +insert into t2 values (); +rename table t2 to t22; +insert into t22 values (); +select * from t22; +id +1 +2 +create table t3 (id int key auto_increment) auto_id_cache 100; +insert into t3 values (); +rename table t3 to t33; +insert into t33 values (); +select * from t33; +id +1 +2 +drop table if exists t0; +create table t0 (id int auto_increment,k int,c char(120)) ; +drop table if exists t1; +create table t1 (id int auto_increment,k int,c char(120)) engine = MyISAM; +drop table if exists t2; +create table t2 (id int auto_increment,k int,c char(120)) engine = InnoDB; +drop table if exists t3; +create table t3 (id int auto_increment,k int,c char(120)) auto_id_cache 1; +drop table if exists t4; +create table t4 (id int auto_increment,k int,c char(120)) auto_id_cache 100; +drop table if exists t5; +create table t5 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) ; +drop table if exists t6; +create table t6 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) engine = MyISAM; +drop table if exists t7; +create table t7 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) engine = InnoDB; +drop table if exists t8; +create table t8 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) auto_id_cache 1; +drop table if exists t9; +create table t9 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) auto_id_cache 100; +drop table if exists t10; +create table t10 (id int auto_increment,k int,c char(120),key idx_1(id)) ; +drop table if exists t11; +create table t11 (id int auto_increment,k int,c char(120),key idx_1(id)) engine = MyISAM; +drop table if exists t12; +create table t12 (id int auto_increment,k int,c char(120),key idx_1(id)) engine = InnoDB; +drop table if exists t13; +create table t13 (id int auto_increment,k int,c char(120),key idx_1(id)) auto_id_cache 1; +drop table if exists t14; +create table t14 (id int auto_increment,k int,c char(120),key idx_1(id)) auto_id_cache 100; +drop table if exists t15; +create table t15 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) ; +drop table if exists t16; +create table t16 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) engine = MyISAM; +drop table if exists t17; +create table t17 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) engine = InnoDB; +drop table if exists t18; +create table t18 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) auto_id_cache 1; +drop table if exists t19; +create table t19 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) auto_id_cache 100; +create table tt1 (id int); +alter table tt1 add column (c int auto_increment); +Error 8200 (HY000): unsupported add column 'c' constraint AUTO_INCREMENT when altering 'executor__autoid.tt1' +create table tt2 (id int, c int auto_increment, key c_idx(c)); +alter table tt2 drop index c_idx; +drop table if exists t_473; +create table t_473 (id int key auto_increment); +insert into t_473 values (); +select * from t_473; +id +1 +show table t_473 next_row_id; +DB_NAME TABLE_NAME COLUMN_NAME NEXT_GLOBAL_ROW_ID ID_TYPE +executor__autoid t_473 id 30001 _TIDB_ROWID +executor__autoid t_473 id 1 AUTO_INCREMENT +alter table t_473 auto_id_cache = 100; +show table t_473 next_row_id; +DB_NAME TABLE_NAME COLUMN_NAME NEXT_GLOBAL_ROW_ID ID_TYPE +executor__autoid t_473 id 30001 _TIDB_ROWID +executor__autoid t_473 id 1 AUTO_INCREMENT +insert into t_473 values (); +select * from t_473; +id +1 +30001 +show table t_473 next_row_id; +DB_NAME TABLE_NAME COLUMN_NAME NEXT_GLOBAL_ROW_ID ID_TYPE +executor__autoid t_473 id 30101 _TIDB_ROWID +executor__autoid t_473 id 1 AUTO_INCREMENT +alter table t_473 auto_id_cache = 1; +Error 1105 (HY000): Can't Alter AUTO_ID_CACHE between 1 and non-1, the underlying implementation is different +drop table if exists io; +set auto_increment_offset = 10; +set auto_increment_increment = 5; +create table io (a int key auto_increment); +insert into io values (null),(null),(null); +select * from io; +a +10 +15 +20 +drop table io; +create table io (a int key auto_increment) AUTO_ID_CACHE 1; +insert into io values (null),(null),(null); +select * from io; +a +10 +15 +20 +drop table io; +create table io (a int key auto_increment); +set auto_increment_offset = 10; +set auto_increment_increment = 2; +insert into io values (),(),(); +select * from io; +a +10 +12 +14 +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +a +15 +20 +25 +delete from io; +set auto_increment_increment = 10; +insert into io values (),(),(); +select * from io; +a +30 +40 +50 +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +a +55 +60 +65 +drop table io; +create table io (a int key auto_increment) AUTO_ID_CACHE 1; +set auto_increment_offset = 10; +set auto_increment_increment = 2; +insert into io values (),(),(); +select * from io; +a +10 +12 +14 +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +a +15 +20 +25 +delete from io; +set auto_increment_increment = 10; +insert into io values (),(),(); +select * from io; +a +30 +40 +50 +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +a +55 +60 +65 +drop table io; +set auto_increment_offset = 10; +set auto_increment_increment = 2; +create table io (a int, b int auto_increment, key(b)); +insert into io(b) values (null),(null),(null); +select b from io; +b +10 +12 +14 +select _tidb_rowid from io; +_tidb_rowid +15 +16 +17 +delete from io; +set auto_increment_increment = 10; +insert into io(b) values (null),(null),(null); +select b from io; +b +20 +30 +40 +select _tidb_rowid from io; +_tidb_rowid +41 +42 +43 +drop table io; +set auto_increment_offset = 10; +set auto_increment_increment = 2; +create table io (a int, b int auto_increment, key(b)) AUTO_ID_CACHE 1; +insert into io(b) values (null),(null),(null); +select b from io; +b +10 +12 +14 +select _tidb_rowid from io; +_tidb_rowid +1 +2 +3 +delete from io; +set auto_increment_increment = 10; +insert into io(b) values (null),(null),(null); +select b from io; +b +20 +30 +40 +select _tidb_rowid from io; +_tidb_rowid +4 +5 +6 +drop table io; +set auto_increment_offset = -1; +show warnings; +Level Code Message +Warning 1292 Truncated incorrect auto_increment_offset value: '-1' +set auto_increment_increment = -2; +show warnings; +Level Code Message +Warning 1292 Truncated incorrect auto_increment_increment value: '-2' +show variables like 'auto_increment%'; +Variable_name Value +auto_increment_increment 1 +auto_increment_offset 1 +set auto_increment_offset = 65536; +show warnings; +Level Code Message +Warning 1292 Truncated incorrect auto_increment_offset value: '65536' +set auto_increment_increment = 65536; +show warnings; +Level Code Message +Warning 1292 Truncated incorrect auto_increment_increment value: '65536' +show variables like 'auto_increment%'; +Variable_name Value +auto_increment_increment 65535 +auto_increment_offset 65535 +set auto_increment_offset = default; +set auto_increment_increment = default; diff --git a/tests/integrationtest/t/executor/autoid.test b/tests/integrationtest/t/executor/autoid.test new file mode 100644 index 0000000000000..7adb3104caf80 --- /dev/null +++ b/tests/integrationtest/t/executor/autoid.test @@ -0,0 +1,485 @@ +# TestAutoIncrementInsertMinMax +drop table if exists t0, t1, t2, t3, t4, t5, t6, t7, t8, t9; +create table t0 (a tinyint signed key auto_increment) ; +insert into t0 values (-128); +insert into t0 values (); +insert into t0 values (0); +insert into t0 values (); +insert into t0 values (127); +-- error 1690 +insert into t0 values (); +select * from t0 order by a; +drop table t0; +create table t1 (a tinyint unsigned key auto_increment) ; +insert into t1 values (0); +insert into t1 values (); +insert into t1 values (127); +insert into t1 values (); +insert into t1 values (255); +-- error 1690 +insert into t1 values (); +select * from t1 order by a; +drop table t1; +create table t2 (a smallint signed key auto_increment) ; +insert into t2 values (-32768); +insert into t2 values (); +insert into t2 values (0); +insert into t2 values (); +insert into t2 values (32767); +-- error 1690 +insert into t2 values (); +select * from t2 order by a; +drop table t2; +create table t3 (a smallint unsigned key auto_increment) ; +insert into t3 values (0); +insert into t3 values (); +insert into t3 values (32767); +insert into t3 values (); +insert into t3 values (65535); +-- error 1690 +insert into t3 values (); +select * from t3 order by a; +drop table t3; +create table t4 (a mediumint signed key auto_increment) ; +insert into t4 values (-8388608); +insert into t4 values (); +insert into t4 values (0); +insert into t4 values (); +insert into t4 values (8388607); +-- error 1690 +insert into t4 values (); +select * from t4 order by a; +drop table t4; +create table t5 (a mediumint unsigned key auto_increment) ; +insert into t5 values (0); +insert into t5 values (); +insert into t5 values (8388607); +insert into t5 values (); +insert into t5 values (16777215); +-- error 1690 +insert into t5 values (); +select * from t5 order by a; +drop table t5; +create table t6 (a integer signed key auto_increment) ; +insert into t6 values (-2147483648); +insert into t6 values (); +insert into t6 values (0); +insert into t6 values (); +insert into t6 values (2147483647); +-- error 1690 +insert into t6 values (); +select * from t6 order by a; +drop table t6; +create table t7 (a integer unsigned key auto_increment) ; +insert into t7 values (0); +insert into t7 values (); +insert into t7 values (2147483647); +insert into t7 values (); +insert into t7 values (4294967295); +-- error 1690 +insert into t7 values (); +select * from t7 order by a; +drop table t7; +create table t8 (a bigint signed key auto_increment) ; +insert into t8 values (-9223372036854775808); +insert into t8 values (); +insert into t8 values (0); +insert into t8 values (); +insert into t8 values (9223372036854775807); +-- error 1467 +insert into t8 values (); +select * from t8 order by a; +drop table t8; +create table t9 (a bigint unsigned key auto_increment) ; +insert into t9 values (0); +insert into t9 values (); +insert into t9 values (9223372036854775807); +insert into t9 values (); +select * from t9 order by a; +drop table t9; +create table t0 (a tinyint signed key auto_increment) auto_id_cache 1; +insert into t0 values (-128); +insert into t0 values (); +insert into t0 values (0); +insert into t0 values (); +insert into t0 values (127); +-- error 1690 +insert into t0 values (); +select * from t0 order by a; +drop table t0; +create table t1 (a tinyint unsigned key auto_increment) auto_id_cache 1; +insert into t1 values (0); +insert into t1 values (); +insert into t1 values (127); +insert into t1 values (); +insert into t1 values (255); +-- error 1690 +insert into t1 values (); +select * from t1 order by a; +drop table t1; +create table t2 (a smallint signed key auto_increment) auto_id_cache 1; +insert into t2 values (-32768); +insert into t2 values (); +insert into t2 values (0); +insert into t2 values (); +insert into t2 values (32767); +-- error 1690 +insert into t2 values (); +select * from t2 order by a; +drop table t2; +create table t3 (a smallint unsigned key auto_increment) auto_id_cache 1; +insert into t3 values (0); +insert into t3 values (); +insert into t3 values (32767); +insert into t3 values (); +insert into t3 values (65535); +-- error 1690 +insert into t3 values (); +select * from t3 order by a; +drop table t3; +create table t4 (a mediumint signed key auto_increment) auto_id_cache 1; +insert into t4 values (-8388608); +insert into t4 values (); +insert into t4 values (0); +insert into t4 values (); +insert into t4 values (8388607); +-- error 1690 +insert into t4 values (); +select * from t4 order by a; +drop table t4; +create table t5 (a mediumint unsigned key auto_increment) auto_id_cache 1; +insert into t5 values (0); +insert into t5 values (); +insert into t5 values (8388607); +insert into t5 values (); +insert into t5 values (16777215); +-- error 1690 +insert into t5 values (); +select * from t5 order by a; +drop table t5; +create table t6 (a integer signed key auto_increment) auto_id_cache 1; +insert into t6 values (-2147483648); +insert into t6 values (); +insert into t6 values (0); +insert into t6 values (); +insert into t6 values (2147483647); +-- error 1690 +insert into t6 values (); +select * from t6 order by a; +drop table t6; +create table t7 (a integer unsigned key auto_increment) auto_id_cache 1; +insert into t7 values (0); +insert into t7 values (); +insert into t7 values (2147483647); +insert into t7 values (); +insert into t7 values (4294967295); +-- error 1690 +insert into t7 values (); +select * from t7 order by a; +drop table t7; +create table t8 (a bigint signed key auto_increment) auto_id_cache 1; +insert into t8 values (-9223372036854775808); +insert into t8 values (); +insert into t8 values (0); +insert into t8 values (); +insert into t8 values (9223372036854775807); +-- error 1105 +insert into t8 values (); +select * from t8 order by a; +drop table t8; +create table t9 (a bigint unsigned key auto_increment) auto_id_cache 1; +insert into t9 values (0); +insert into t9 values (); +insert into t9 values (9223372036854775807); +insert into t9 values (); +select * from t9 order by a; +drop table t9; +create table t0 (a tinyint signed key auto_increment) auto_id_cache 100; +insert into t0 values (-128); +insert into t0 values (); +insert into t0 values (0); +insert into t0 values (); +insert into t0 values (127); +-- error 1690 +insert into t0 values (); +select * from t0 order by a; +drop table t0; +create table t1 (a tinyint unsigned key auto_increment) auto_id_cache 100; +insert into t1 values (0); +insert into t1 values (); +insert into t1 values (127); +insert into t1 values (); +insert into t1 values (255); +-- error 1690 +insert into t1 values (); +select * from t1 order by a; +drop table t1; +create table t2 (a smallint signed key auto_increment) auto_id_cache 100; +insert into t2 values (-32768); +insert into t2 values (); +insert into t2 values (0); +insert into t2 values (); +insert into t2 values (32767); +-- error 1690 +insert into t2 values (); +select * from t2 order by a; +drop table t2; +create table t3 (a smallint unsigned key auto_increment) auto_id_cache 100; +insert into t3 values (0); +insert into t3 values (); +insert into t3 values (32767); +insert into t3 values (); +insert into t3 values (65535); +-- error 1690 +insert into t3 values (); +select * from t3 order by a; +drop table t3; +create table t4 (a mediumint signed key auto_increment) auto_id_cache 100; +insert into t4 values (-8388608); +insert into t4 values (); +insert into t4 values (0); +insert into t4 values (); +insert into t4 values (8388607); +-- error 1690 +insert into t4 values (); +select * from t4 order by a; +drop table t4; +create table t5 (a mediumint unsigned key auto_increment) auto_id_cache 100; +insert into t5 values (0); +insert into t5 values (); +insert into t5 values (8388607); +insert into t5 values (); +insert into t5 values (16777215); +-- error 1690 +insert into t5 values (); +select * from t5 order by a; +drop table t5; +create table t6 (a integer signed key auto_increment) auto_id_cache 100; +insert into t6 values (-2147483648); +insert into t6 values (); +insert into t6 values (0); +insert into t6 values (); +insert into t6 values (2147483647); +-- error 1690 +insert into t6 values (); +select * from t6 order by a; +drop table t6; +create table t7 (a integer unsigned key auto_increment) auto_id_cache 100; +insert into t7 values (0); +insert into t7 values (); +insert into t7 values (2147483647); +insert into t7 values (); +insert into t7 values (4294967295); +-- error 1690 +insert into t7 values (); +select * from t7 order by a; +drop table t7; +create table t8 (a bigint signed key auto_increment) auto_id_cache 100; +insert into t8 values (-9223372036854775808); +insert into t8 values (); +insert into t8 values (0); +insert into t8 values (); +insert into t8 values (9223372036854775807); +-- error 1467 +insert into t8 values (); +select * from t8 order by a; +drop table t8; +create table t9 (a bigint unsigned key auto_increment) auto_id_cache 100; +insert into t9 values (0); +insert into t9 values (); +insert into t9 values (9223372036854775807); +insert into t9 values (); +select * from t9 order by a; +drop table t9; +create table t10 (a integer key auto_increment) auto_id_cache 1; +-- error 1264 +insert into t10 values (2147483648); +-- error 1264 +insert into t10 values (-2147483649); + +# TestRenameTableForAutoIncrement +drop table if exists t1, t2, t3, t11, t22, t33; +create table t1 (id int key auto_increment); +insert into t1 values (); +rename table t1 to t11; +insert into t11 values (); +select * from t11; + +## auto_id_cache 1 use another implementation and do not have such bug. +create table t2 (id int key auto_increment) auto_id_cache 1; +insert into t2 values (); +rename table t2 to t22; +insert into t22 values (); +select * from t22; + +create table t3 (id int key auto_increment) auto_id_cache 100; +insert into t3 values (); +rename table t3 to t33; +insert into t33 values (); +select * from t33; + +# TestAutoIDConstraint +# Remove the constraint that auto id column must be defined as a key +# See https://github.com/pingcap/tidb/issues/40580 +drop table if exists t0; +create table t0 (id int auto_increment,k int,c char(120)) ; +drop table if exists t1; +create table t1 (id int auto_increment,k int,c char(120)) engine = MyISAM; +drop table if exists t2; +create table t2 (id int auto_increment,k int,c char(120)) engine = InnoDB; +drop table if exists t3; +create table t3 (id int auto_increment,k int,c char(120)) auto_id_cache 1; +drop table if exists t4; +create table t4 (id int auto_increment,k int,c char(120)) auto_id_cache 100; +drop table if exists t5; +create table t5 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) ; +drop table if exists t6; +create table t6 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) engine = MyISAM; +drop table if exists t7; +create table t7 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) engine = InnoDB; +drop table if exists t8; +create table t8 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) auto_id_cache 1; +drop table if exists t9; +create table t9 (id int auto_increment,k int,c char(120),PRIMARY KEY(k, id)) auto_id_cache 100; +drop table if exists t10; +create table t10 (id int auto_increment,k int,c char(120),key idx_1(id)) ; +drop table if exists t11; +create table t11 (id int auto_increment,k int,c char(120),key idx_1(id)) engine = MyISAM; +drop table if exists t12; +create table t12 (id int auto_increment,k int,c char(120),key idx_1(id)) engine = InnoDB; +drop table if exists t13; +create table t13 (id int auto_increment,k int,c char(120),key idx_1(id)) auto_id_cache 1; +drop table if exists t14; +create table t14 (id int auto_increment,k int,c char(120),key idx_1(id)) auto_id_cache 100; +drop table if exists t15; +create table t15 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) ; +drop table if exists t16; +create table t16 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) engine = MyISAM; +drop table if exists t17; +create table t17 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) engine = InnoDB; +drop table if exists t18; +create table t18 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) auto_id_cache 1; +drop table if exists t19; +create table t19 (id int auto_increment,k int,c char(120),PRIMARY KEY(`k`, `id`), key idx_1(id)) auto_id_cache 100; + +## alter table add auto id column is not supported, but cover it here to prevent regression +create table tt1 (id int); +-- error 8200 +alter table tt1 add column (c int auto_increment); + +## Cover case: create table with auto id column as key, and remove it later +create table tt2 (id int, c int auto_increment, key c_idx(c)); +alter table tt2 drop index c_idx; + +# TestAlterTableAutoIDCache +drop table if exists t_473; +create table t_473 (id int key auto_increment); +insert into t_473 values (); +select * from t_473; +show table t_473 next_row_id; +alter table t_473 auto_id_cache = 100; +show table t_473 next_row_id; +insert into t_473 values (); +select * from t_473; +show table t_473 next_row_id; + +## 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. +-- error 1105 +alter table t_473 auto_id_cache = 1; + +# TestAutoIDIncrementAndOffset There is a potential issue in MySQL: when the value of auto_increment_offset is greater +# than that of auto_increment_increment, the value of auto_increment_offset is ignored +# (https://dev.mysql.com/doc/refman/8.0/en/replication-options-master.html#sysvar_auto_increment_increment), +# This issue is a flaw of the implementation of MySQL and it doesn't exist in TiDB. +drop table if exists io; +set auto_increment_offset = 10; +set auto_increment_increment = 5; + +create table io (a int key auto_increment); +insert into io values (null),(null),(null); +select * from io; +drop table io; +create table io (a int key auto_increment) AUTO_ID_CACHE 1; +insert into io values (null),(null),(null); +select * from io; +drop table io; + +## Test handle is PK. +create table io (a int key auto_increment); +set auto_increment_offset = 10; +set auto_increment_increment = 2; +insert into io values (),(),(); +select * from io; +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +delete from io; +set auto_increment_increment = 10; +insert into io values (),(),(); +select * from io; +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +drop table io; +create table io (a int key auto_increment) AUTO_ID_CACHE 1; +set auto_increment_offset = 10; +set auto_increment_increment = 2; +insert into io values (),(),(); +select * from io; +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +delete from io; +set auto_increment_increment = 10; +insert into io values (),(),(); +select * from io; +delete from io; +set auto_increment_increment = 5; +insert into io values (),(),(); +select * from io; +drop table io; + +## Test handle is not PK. +set auto_increment_offset = 10; +set auto_increment_increment = 2; +create table io (a int, b int auto_increment, key(b)); +insert into io(b) values (null),(null),(null); +select b from io; +select _tidb_rowid from io; +delete from io; +set auto_increment_increment = 10; +insert into io(b) values (null),(null),(null); +select b from io; +select _tidb_rowid from io; +drop table io; +set auto_increment_offset = 10; +set auto_increment_increment = 2; +create table io (a int, b int auto_increment, key(b)) AUTO_ID_CACHE 1; +insert into io(b) values (null),(null),(null); +select b from io; +select _tidb_rowid from io; +delete from io; +set auto_increment_increment = 10; +insert into io(b) values (null),(null),(null); +select b from io; +select _tidb_rowid from io; +drop table io; + +set auto_increment_offset = -1; +show warnings; +set auto_increment_increment = -2; +show warnings; +show variables like 'auto_increment%'; + +set auto_increment_offset = 65536; +show warnings; +set auto_increment_increment = 65536; +show warnings; +show variables like 'auto_increment%'; + +set auto_increment_offset = default; +set auto_increment_increment = default;