diff --git a/ddl/multi_schema_change.go b/ddl/multi_schema_change.go index 19c355ebfa8da..8097ad9798664 100644 --- a/ddl/multi_schema_change.go +++ b/ddl/multi_schema_change.go @@ -110,6 +110,7 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve subJobs := make([]model.SubJob, len(job.MultiSchemaInfo.SubJobs)) // Step the sub-jobs to the non-revertible states all at once. // We only generate 1 schema version for these sub-job. + actionTypes := make([]model.ActionType, 0, len(job.MultiSchemaInfo.SubJobs)) for i, sub := range job.MultiSchemaInfo.SubJobs { if sub.IsFinished() { continue @@ -127,13 +128,36 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve sub.FromProxyJob(&proxyJob, proxyJobVer) if err != nil || proxyJob.Error != nil { for j := i - 1; j >= 0; j-- { + // TODO if some sub-job is finished, this will empty them + // also some sub-job cannot be rollback completely, maybe keep them? job.MultiSchemaInfo.SubJobs[j] = &subJobs[j] } handleRevertibleException(job, sub, proxyJob.Error) // The TableInfo and sub-jobs should be restored // because some schema changes update the transaction aggressively. + // TODO this error handling cannot handle below case: + // suppose the job is for "alter table t auto_increment = 100, add column c int". + // if we fail on "add column c int", the allocator is rebased to 100 + // which cannot be rollback, but it's table-info.AutoIncID is rollback by below call. + // TODO we should also change schema diff of 'ver' if len(actionTypes) > 1. return updateVersionAndTableInfo(d, t, job, tblInfo, true) } + actionTypes = append(actionTypes, sub.Type) + } + if len(actionTypes) > 1 { + // only single table schema changes can be put into a multi-schema-change + // job except AddForeignKey which is handled separately in the first loop. + // so this diff is enough, but it wound be better to accumulate all the diffs, + // and then merge them into a single diff. + if err = t.SetSchemaDiff(&model.SchemaDiff{ + Version: ver, + Type: job.Type, + TableID: job.TableID, + SchemaID: job.SchemaID, + SubActionTypes: actionTypes, + }); err != nil { + return ver, err + } } // All the sub-jobs are non-revertible. job.MarkNonRevertible() diff --git a/infoschema/builder.go b/infoschema/builder.go index daf5acf3e4da6..d7c9ad926ad36 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -488,6 +488,10 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 b.copySortedTables(oldTableID, newTableID) tblIDs := make([]int64, 0, 2) +<<<<<<< HEAD:infoschema/builder.go +======= + var keptAllocs autoid.Allocators +>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go // We try to reuse the old allocator, so the cached auto ID can be reused. var allocs autoid.Allocators if tableIDIsValid(oldTableID) { @@ -499,15 +503,27 @@ 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 { +<<<<<<< HEAD:infoschema/builder.go oldAllocs, _ := b.is.AllocByID(oldTableID) allocs = filterAllocators(diff, oldAllocs) +======= + // 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, _ := allocByID(b, oldTableID) + keptAllocs = getKeptAllocators(diff, oldAllocs) +>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go } tmpIDs := tblIDs if (diff.Type == model.ActionRenameTable || diff.Type == model.ActionRenameTables) && diff.OldSchemaID != diff.SchemaID { oldRoDBInfo, ok := b.is.SchemaByID(diff.OldSchemaID) if !ok { +<<<<<<< HEAD:infoschema/builder.go return nil, ErrDatabaseNotExists.GenWithStackByArgs( +======= + return nil, keptAllocs, ErrDatabaseNotExists.GenWithStackByArgs( +>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go fmt.Sprintf("(Schema ID %d)", diff.OldSchemaID), ) } @@ -522,6 +538,29 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 tblIDs = tmpIDs } } +<<<<<<< HEAD:infoschema/builder.go +======= + return tblIDs, keptAllocs, nil +} + +func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int64, error) { + roDBInfo, ok := b.infoSchema.SchemaByID(diff.SchemaID) + if !ok { + return nil, ErrDatabaseNotExists.GenWithStackByArgs( + fmt.Sprintf("(Schema ID %d)", diff.SchemaID), + ) + } + dbInfo := b.getSchemaAndCopyIfNecessary(roDBInfo.Name.L) + oldTableID, newTableID := b.getTableIDs(diff) + b.updateBundleForTableUpdate(diff, newTableID, oldTableID) + b.copySortedTables(oldTableID, newTableID) + + tblIDs, allocs, err := dropTableForUpdate(b, newTableID, oldTableID, dbInfo, diff) + if err != nil { + return nil, err + } + +>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go if tableIDIsValid(newTableID) { // All types except DropTableOrView. var err error @@ -533,16 +572,33 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 return tblIDs, nil } -func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators { - var newAllocs autoid.Allocators +// getKeptAllocators get allocators that is not changed by the DDL. +func getKeptAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators { + var autoIDChanged, autoRandomChanged bool switch diff.Type { case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: + autoIDChanged = true + case model.ActionRebaseAutoRandomBase: + autoRandomChanged = true + case model.ActionMultiSchemaChange: + for _, t := range diff.SubActionTypes { + switch t { + case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache: + autoIDChanged = true + case model.ActionRebaseAutoRandomBase: + autoRandomChanged = true + } + } + } + var newAllocs autoid.Allocators + switch { + case autoIDChanged: // Only drop auto-increment allocator. newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool { tp := a.GetType() return tp != autoid.RowIDAllocType && tp != autoid.AutoIncrementType }) - case model.ActionRebaseAutoRandomBase: + case autoRandomChanged: // Only drop auto-random allocator. newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool { tp := a.GetType() diff --git a/infoschema/builder_test.go b/infoschema/builder_test.go new file mode 100644 index 0000000000000..cd9d34c45bcfd --- /dev/null +++ b/infoschema/builder_test.go @@ -0,0 +1,94 @@ +// Copyright 2024 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 infoschema + +import ( + "fmt" + "testing" + + "github.com/pingcap/tidb/pkg/meta/autoid" + "github.com/pingcap/tidb/pkg/parser/model" + "github.com/stretchr/testify/require" +) + +type mockAlloc struct { + autoid.Allocator + tp autoid.AllocatorType +} + +func (m *mockAlloc) GetType() autoid.AllocatorType { + return m.tp +} + +func TestGetKeptAllocators(t *testing.T) { + checkAllocators := func(allocators autoid.Allocators, expected []autoid.AllocatorType) { + require.Len(t, allocators.Allocs, len(expected)) + for i, tp := range expected { + require.Equal(t, tp, allocators.Allocs[i].GetType()) + } + } + allocators := autoid.Allocators{Allocs: []autoid.Allocator{ + &mockAlloc{tp: autoid.RowIDAllocType}, + &mockAlloc{tp: autoid.AutoIncrementType}, + &mockAlloc{tp: autoid.AutoRandomType}, + }} + cases := []struct { + diff *model.SchemaDiff + expected []autoid.AllocatorType + }{ + { + diff: &model.SchemaDiff{Type: model.ActionTruncateTable}, + expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType, autoid.AutoRandomType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionRebaseAutoID}, + expected: []autoid.AllocatorType{autoid.AutoRandomType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionModifyTableAutoIdCache}, + expected: []autoid.AllocatorType{autoid.AutoRandomType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionRebaseAutoRandomBase}, + expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange, + SubActionTypes: []model.ActionType{model.ActionAddColumn, model.ActionRebaseAutoID}}, + expected: []autoid.AllocatorType{autoid.AutoRandomType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange, + SubActionTypes: []model.ActionType{model.ActionModifyTableAutoIdCache}}, + expected: []autoid.AllocatorType{autoid.AutoRandomType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange, + SubActionTypes: []model.ActionType{model.ActionRebaseAutoRandomBase}}, + expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType}, + }, + { + diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange, + SubActionTypes: []model.ActionType{model.ActionAddColumn}}, + expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType, autoid.AutoRandomType}, + }, + } + for i, c := range cases { + t.Run(fmt.Sprintf("case %d", i), func(t *testing.T) { + res := getKeptAllocators(c.diff, allocators) + checkAllocators(res, c.expected) + }) + } +} diff --git a/parser/model/ddl.go b/parser/model/ddl.go index 297c57f0c8602..1552cedcd7c89 100644 --- a/parser/model/ddl.go +++ b/parser/model/ddl.go @@ -998,6 +998,12 @@ type SchemaDiff struct { SchemaID int64 `json:"schema_id"` TableID int64 `json:"table_id"` + // SubActionTypes is the list of action types done together within a multiple schema + // change job. As the job might contain multiple steps that changes schema version, + // if some step only contains one action, Type will be that action, and SubActionTypes + // will be empty. + // for other types of job, it will always be empty. + SubActionTypes []ActionType `json:"sub_action_types,omitempty"` // OldTableID is the table ID before truncate, only used by truncate table DDL. OldTableID int64 `json:"old_table_id"` // OldSchemaID is the schema ID before rename table, only used by rename table DDL. diff --git a/pkg/infoschema/BUILD.bazel b/pkg/infoschema/BUILD.bazel new file mode 100644 index 0000000000000..84aa4859643cb --- /dev/null +++ b/pkg/infoschema/BUILD.bazel @@ -0,0 +1,116 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test") + +go_library( + name = "infoschema", + srcs = [ + "builder.go", + "builder_misc.go", + "bundle_builder.go", + "cache.go", + "cluster.go", + "error.go", + "infoschema.go", + "infoschema_v2.go", + "interface.go", + "metric_table_def.go", + "metrics_schema.go", + "sieve.go", + "tables.go", + ], + importpath = "github.com/pingcap/tidb/pkg/infoschema", + visibility = ["//visibility:public"], + deps = [ + "//pkg/config", + "//pkg/ddl/placement", + "//pkg/ddl/resourcegroup", + "//pkg/domain/infosync", + "//pkg/errno", + "//pkg/infoschema/context", + "//pkg/infoschema/internal", + "//pkg/infoschema/metrics", + "//pkg/kv", + "//pkg/meta", + "//pkg/meta/autoid", + "//pkg/parser/auth", + "//pkg/parser/charset", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/parser/terror", + "//pkg/privilege", + "//pkg/session/txninfo", + "//pkg/sessionctx", + "//pkg/sessionctx/variable", + "//pkg/table", + "//pkg/table/tables", + "//pkg/types", + "//pkg/util", + "//pkg/util/dbterror", + "//pkg/util/deadlockhistory", + "//pkg/util/domainutil", + "//pkg/util/execdetails", + "//pkg/util/intest", + "//pkg/util/logutil", + "//pkg/util/mock", + "//pkg/util/sem", + "//pkg/util/set", + "//pkg/util/size", + "//pkg/util/stmtsummary", + "@com_github_ngaut_pools//:pools", + "@com_github_pingcap_errors//:errors", + "@com_github_pingcap_failpoint//:failpoint", + "@com_github_pingcap_kvproto//pkg/diagnosticspb", + "@com_github_pingcap_kvproto//pkg/metapb", + "@com_github_pingcap_log//:log", + "@com_github_tidwall_btree//:btree", + "@com_github_tikv_client_go_v2//tikv", + "@com_github_tikv_pd_client//http", + "@org_golang_google_grpc//:grpc", + "@org_golang_google_grpc//credentials", + "@org_golang_google_grpc//credentials/insecure", + "@org_golang_x_sync//singleflight", + "@org_uber_go_zap//:zap", + ], +) + +go_test( + name = "infoschema_test", + timeout = "short", + srcs = [ + "bench_test.go", + "builder_test.go", + "infoschema_test.go", + "infoschema_v2_test.go", + "main_test.go", + "metrics_schema_test.go", + "sieve_test.go", + ], + embed = [":infoschema"], + flaky = True, + shard_count = 20, + deps = [ + "//pkg/ddl/placement", + "//pkg/domain", + "//pkg/infoschema/internal", + "//pkg/kv", + "//pkg/meta", + "//pkg/meta/autoid", + "//pkg/parser/model", + "//pkg/parser/mysql", + "//pkg/sessionctx/variable", + "//pkg/store/driver", + "//pkg/table", + "//pkg/testkit", + "//pkg/testkit/testsetup", + "//pkg/testkit/testutil", + "//pkg/types", + "//pkg/util", + "//pkg/util/logutil", + "//pkg/util/set", + "//pkg/util/size", + "@com_github_pingcap_errors//:errors", + "@com_github_prometheus_prometheus//promql/parser", + "@com_github_stretchr_testify//require", + "@org_uber_go_goleak//:goleak", + "@org_uber_go_zap//:zap", + ], +) diff --git a/tests/integrationtest/r/ddl/multi_schema_change.result b/tests/integrationtest/r/ddl/multi_schema_change.result new file mode 100644 index 0000000000000..f59048933d130 --- /dev/null +++ b/tests/integrationtest/r/ddl/multi_schema_change.result @@ -0,0 +1,438 @@ +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column b int default 2, add column c int default 3; +select * from t; +a b c +1 2 3 +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (b int default 2, c int default 3); +select * from t; +a b c +1 2 3 +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t add column (d int default 4, e int default 5); +select * from t; +a b c d e +1 2 3 4 5 +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (index i(a), index i1(a)); +select * from t use index (i, i1); +a +1 +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (b int default 2, index i(a), primary key (a)); +select * from t use index (i, primary); +a b +1 2 +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (index i(a)); +select * from t use index (i); +a +1 +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t add column (index i1(a, b, c), index i2(c, b, a), index i3((a + 1)), index i4((c - 1))); +select * from t use index (i1, i2); +a b c +1 2 3 +admin check table t; +drop table if exists t; +create table t (a int default 1); +insert into t values (); +alter table t add column if not exists (b int default 2, c int default 3); +select * from t; +a b c +1 2 3 +alter table t add column if not exists (c int default 3, d int default 4); +show warnings; +Level Code Message +Note 1060 Duplicate column name 'c' +select * from t; +a b c d +1 2 3 4 +drop table if exists t; +create table t (a int); +alter table t add column b int after a, add column c int after b; +Error 1054 (42S22): Unknown column 'b' in 't' +alter table t add column c int after b, add column b int; +Error 1054 (42S22): Unknown column 'b' in 't' +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t +add column d int default 4 first, +add column e int default 5 after b, +add column f int default 6 after b; +select * from t; +d a b f e c +4 1 2 6 5 3 +drop table if exists t; +create table t (a int default 1); +insert into t values (); +alter table t add column b int default 2, add column if not exists a int; +show warnings; +Level Code Message +Note 1060 Duplicate column name 'a' +select * from t; +a b +1 2 +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t add column c double default 3.0, add column d double as (a + b); +select * from t; +a b c d +1 2 3 3 +drop table if exists t; +create table t (a int default 1, c int default 4); +alter table t add column b int default 2, add column b int default 3; +Error 8200 (HY000): Unsupported operate same column 'b' +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t modify column b double, add column c double as (a + b); +Error 8200 (HY000): Unsupported operate same column 'b' +drop table if exists t; +create table t (a int, b int); +alter table t drop column a, drop column b; +Error 1090 (42000): You can't delete all columns with ALTER TABLE; use DROP TABLE instead +drop table if exists t; +create table t (a int, b int, c int, d int, e int); +insert into t values (1, 2, 3, 4, 5); +alter table t drop column a, drop column d, drop column b; +select * from t; +c e +3 5 +drop table if exists t; +create table t (a int default 1, c int default 4); +alter table t drop column a, drop column a; +Error 8200 (HY000): Unsupported operate same column 'a' +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t drop column if exists c, drop column a; +show warnings; +Level Code Message +Note 1091 Can't DROP 'c'; check that column/key exists +select * from t; +b +2 +drop table if exists t; +create table t (a int default 1, b int default 2, c int default 3); +insert into t values (); +alter table t drop column a, drop column if exists d, drop column c; +show warnings; +Level Code Message +Note 1091 Can't DROP 'd'; check that column/key exists +select * from t; +b +2 +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t add column c int default 3, drop column a, add column d int default 4, drop column b; +select * from t; +c d +3 4 +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t drop column a, drop column b, add column c int default 3, add column d int default 4; +select * from t; +c d +3 4 +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t add column c int default 3 after a, add column d int default 4 first, drop column a, drop column b; +Error 8200 (HY000): Unsupported operate same column 'a' +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t add index t(a, b), add index t1(a); +alter table t add index t2(a), add index t3(a, b); +select * from t use index (t, t1, t2, t3); +a b c +1 2 3 +admin check table t; +drop table if exists t; +create table t (a int, b int, c int); +alter table t add index t(a), add index t(b); +Error 8200 (HY000): Unsupported operate same index 't' +show index from t; +Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Visible Expression Clustered +drop table if exists t; +create table t (a int, b int, c int); +alter table t add index t(a), drop column a; +Error 8200 (HY000): Unsupported operate same column 'a' +alter table t add index t(a, b), drop column a; +Error 8200 (HY000): Unsupported operate same column 'a' +show index from t; +Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Visible Expression Clustered +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 1, 1), (2, 2, 2), (3, 3, 1); +alter table t add unique index i1(a), add unique index i2(a, b), add unique index i3(c); +Error 1062 (23000): Duplicate entry '1' for key 't.i3' +show index from t; +Table Non_unique Key_name Seq_in_index Column_name Collation Cardinality Sub_part Packed Null Index_type Comment Index_comment Visible Expression Clustered +alter table t add index i1(a), add index i2(a, b), add index i3(c); +drop table if exists t; +create table t (a int, b int, c int, index t(a)); +alter table t drop index t, drop index t; +Error 8200 (HY000): Unsupported operate same index 't' +drop table if exists t; +create table t (id int, c1 int, c2 int, primary key(id) nonclustered, key i1(c1), key i2(c2), key i3(c1, c2)); +insert into t values (1, 2, 3); +alter table t drop index i1, drop index i2; +select * from t use index(i1); +Error 1176 (42000): Key 'i1' doesn't exist in table 't' +select * from t use index(i2); +Error 1176 (42000): Key 'i2' doesn't exist in table 't' +alter table t drop index i3, drop primary key; +select * from t use index(primary); +Error 1176 (42000): Key 'primary' doesn't exist in table 't' +select * from t use index(i3); +Error 1176 (42000): Key 'i3' doesn't exist in table 't' +drop table if exists t; +create table t (a int default 1, b int default 2, c int default 3, index t(a)); +insert into t values (); +alter table t drop index t, drop column a; +select * from t force index(t); +Error 1176 (42000): Key 't' doesn't exist in table 't' +drop table if exists t; +create table t (a int, b int, c int, index t(a)); +alter table t drop index t, add index t(b); +Error 1061 (42000): Duplicate key name 't' +drop table if exists t; +create table t (a int, b int, c int, index t(a)); +alter table t add index t1(b), drop index t1; +Error 1091 (42000): index t1 doesn't exist +drop table if exists t; +create table t (a int, b int, c int, index (a), index(b), index(c)); +insert into t values (1, 2, 3); +alter table t add index xa(a), drop index a, add index xc(c), drop index b, drop index c, add index xb(b); +select * from t use index(xa, xb, xc); +a b c +1 2 3 +select * from t use index(a); +Error 1176 (42000): Key 'a' doesn't exist in table 't' +select * from t use index(b); +Error 1176 (42000): Key 'b' doesn't exist in table 't' +select * from t use index(c); +Error 1176 (42000): Key 'c' doesn't exist in table 't' +admin check table t; +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t modify column a int default 2, modify column a bigint; +Error 8200 (HY000): Unsupported operate same column 'a' +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t modify column b double, drop column b; +Error 8200 (HY000): Unsupported operate same column 'b' +drop table if exists t; +create table t (a int default 1, b int default 2, c int default 3); +insert into t values (); +alter table t modify column b double after c, drop column c; +Error 8200 (HY000): Unsupported operate same column 'c' +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t add index i(a), modify column a int null default 1 after a; +Error 8200 (HY000): Unsupported operate same column 'a' +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t add primary key(a), modify column a int null default 1 after a; +Error 8200 (HY000): Unsupported operate same column 'a' +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t modify column b double, add index idx((a + b)); +Error 8200 (HY000): Unsupported operate same column 'b' +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t modify column b double default 2 after a, add column c int default 3 after a; +select * from t; +a c b +1 3 2 +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t modify column a bigint, modify column b bigint; +insert into t values (9223372036854775807, 9223372036854775807, 1); +select * from t; +a b c +1 2 3 +9223372036854775807 9223372036854775807 1 +drop table if exists t; +create table t (a int, b int, c int, index i1(a), index i2(b), index i3(c), index i4(a, b), index i5(a, b, c)); +insert into t values (1, 2, 3); +alter table t modify column a tinyint, modify column b tinyint, modify column c tinyint; +select * from t; +a b c +1 2 3 +select * from t use index(i1, i2, i3, i4, i5); +a b c +1 2 3 +admin check table t; +drop table if exists t; +create table t (a int, b int, c int, index i1(a), index i2(b), index i3(c), index i4(a, b), index i5(a, b, c)); +insert into t values (1, 2, 3); +alter table t modify column a tinyint after c, modify column b tinyint, modify column c tinyint first; +select * from t; +c b a +3 2 1 +select * from t use index(i1, i2, i3, i4, i5); +c b a +3 2 1 +admin check table t; +drop table if exists t; +create table t (a int, b int, c int, index i1(a), index i2(c, b)); +insert into t values (1, 2, 3), (11, 22, 33); +alter table t modify column b char(255) after c, modify column a bigint; +select * from t; +a c b +1 3 2 +11 33 22 +select * from t use index(i1, i2); +a c b +1 3 2 +11 33 22 +admin check table t; +drop table if exists t; +create table t(a bigint null default '1761233443433596323', index t(a)); +insert into t set a = '-7184819032643664798'; +alter table t change column a b datetime null default '8972-12-24 10:56:03', rename index t to t1; +Error 1292 (22007): Incorrect datetime value: '-7184819032643664798' +drop table if exists t; +create table t (a int, b double, index i(a, b)); +alter table t rename index i to i1, change column b c int; +select count(*) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='i1'; +count(*) +1 +drop table if exists t; +create table t (a int, b double, index i(a, b), index _Idx$_i(a, b)); +alter table t rename index i to i1, change column b c int; +select count(*) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='i1'; +count(*) +1 +select count(*) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='_Idx$_i'; +count(*) +1 +drop table if exists t; +create table t (a int, _Col$_a double, index _Idx$_i(a, _Col$_a), index i(a, _Col$_a)); +alter table t modify column a tinyint; +select count(distinct KEY_NAME) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t'; +count(distinct KEY_NAME) +2 +drop table if exists t; +create table t (a BIGINT NULL DEFAULT '-283977870758975838', b double); +insert into t values (-283977870758975838, 0); +alter table t change column a c tinyint null default '111' after b, modify column b time null default '13:51:02' FIRST; +Error 1690 (22003): constant -283977870758975838 overflows tinyint +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t add index i(b), modify column a int null default 1 after a; +Error 1054 (42S22): Unknown column 'a' in 't' +drop table if exists t; +create table t(a char(3), b int, unique index i1(a), index i2(a, b)); +insert into t values ('aaa', 1), ('aa', 2); +set @@sql_mode = ''; +alter table t modify column b tinyint, modify column a char(2); +Error 1062 (23000): Duplicate entry 'aa' for key 't._Idx$_i1_0' +set @@sql_mode = default; +drop table if exists t; +create table t (a int, b int, c int, index i1(c), index i2(c)); +insert into t values (1, 2, 3); +alter table t add column d int default 4, add index i3(c), drop column a, drop column if exists z, add column if not exists e int default 5, drop index i2, add column f int default 6, drop column b, drop index i1, add column if not exists c int; +select * from t; +c d e f +3 4 5 6 +select * from t use index (i1); +Error 1176 (42000): Key 'i1' doesn't exist in table 't' +select * from t use index (i2); +Error 1176 (42000): Key 'i2' doesn't exist in table 't' +select * from t use index (i3); +c d e f +3 4 5 6 +drop table if exists t; +create table t (a int auto_increment primary key, b int) auto_id_cache = 100; +insert into t(b) values(1); +alter table t modify column b tinyint, auto_increment = 200; +insert into t (b) values (2); +select * from t; +a b +1 1 +200 2 +drop table if exists t; +create table t (a int auto_increment primary key, b int); +alter table t auto_increment = 110, auto_increment = 90; +show warnings; +Level Code Message +Note 1105 Can't reset AUTO_INCREMENT to 90 without FORCE option, using 110 instead +insert into t (b) values (1); +select * from t; +a b +110 1 +drop table if exists t; +create table t (a int, b int) charset = utf8 shard_row_id_bits=2; +alter table t modify column a tinyint, comment = 'abc', charset = utf8mb4; +select TIDB_ROW_ID_SHARDING_INFO, TABLE_COMMENT, TABLE_COLLATION from information_schema.tables where TABLE_SCHEMA='ddl__multi_schema_change' and table_name = 't'; +TIDB_ROW_ID_SHARDING_INFO TABLE_COMMENT TABLE_COLLATION +SHARD_BITS=2 abc utf8mb4_bin +drop table if exists t; +create table t (a tinyint); +insert into t set a = 10; +alter table t add column b int not null, change column a c char(5) first; +select * from t; +c b +10 0 +drop table if exists t; +create table t (a int, b int, index idx(b)); +alter table t add index idx2(a), alter index idx visible; +select * from t use index (idx, idx2); +a b +alter table t drop column b, alter index idx invisible; +Error 1176 (42000): Key 'idx' doesn't exist in table 't' +select a, b from t; +a b +drop table if exists t; +create table t (a int, b int); +alter table t add column c int, auto_id_cache = 10; +Error 8200 (HY000): Unsupported multi schema change for modify auto id cache +drop table if exists t; +CREATE TABLE t (a SMALLINT DEFAULT '30219', b TIME NULL DEFAULT '02:45:06', PRIMARY KEY (a)); +ALTER TABLE t ADD unique INDEX idx4 (b), change column a e MEDIUMINT DEFAULT '5280454' FIRST; +insert ignore into t (e) values (5586359),(501788),(-5961048),(220083),(-4917129),(-7267211),(7750448); +select * from t; +e b +5586359 02:45:06 +admin check table t; +drop table if exists t; +create table t (a int); +insert into t values (123); +alter table t add index i(a), add primary key (a); +show create table t; +Table Create Table +t CREATE TABLE `t` ( + `a` int(11) NOT NULL, + KEY `i` (`a`), + PRIMARY KEY (`a`) /*T![clustered_index] NONCLUSTERED */ +) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_bin diff --git a/tests/integrationtest/t/ddl/multi_schema_change.test b/tests/integrationtest/t/ddl/multi_schema_change.test new file mode 100644 index 0000000000000..3fc412745096d --- /dev/null +++ b/tests/integrationtest/t/ddl/multi_schema_change.test @@ -0,0 +1,367 @@ +# TestMultiSchemaChangeAddColumns +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column b int default 2, add column c int default 3; +select * from t; +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (b int default 2, c int default 3); +select * from t; +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t add column (d int default 4, e int default 5); +select * from t; +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (index i(a), index i1(a)); +select * from t use index (i, i1); +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (b int default 2, index i(a), primary key (a)); +select * from t use index (i, primary); +drop table if exists t; +create table t (a int); +insert into t values (1); +alter table t add column (index i(a)); +select * from t use index (i); +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t add column (index i1(a, b, c), index i2(c, b, a), index i3((a + 1)), index i4((c - 1))); +select * from t use index (i1, i2); +admin check table t; +drop table if exists t; +create table t (a int default 1); +insert into t values (); +alter table t add column if not exists (b int default 2, c int default 3); +select * from t; +alter table t add column if not exists (c int default 3, d int default 4); +show warnings; +select * from t; +drop table if exists t; +create table t (a int); +-- error 1054 +alter table t add column b int after a, add column c int after b; +-- error 1054 +alter table t add column c int after b, add column b int; +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t + add column d int default 4 first, + add column e int default 5 after b, + add column f int default 6 after b; +select * from t; +drop table if exists t; +create table t (a int default 1); +insert into t values (); +alter table t add column b int default 2, add column if not exists a int; +show warnings; +select * from t; +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +alter table t add column c double default 3.0, add column d double as (a + b); +select * from t; +drop table if exists t; +create table t (a int default 1, c int default 4); +-- error 8200 +alter table t add column b int default 2, add column b int default 3; +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +-- error 8200 +alter table t modify column b double, add column c double as (a + b); + +# TestMultiSchemaChangeDropColumns +drop table if exists t; +create table t (a int, b int); +-- error 1090 +alter table t drop column a, drop column b; +drop table if exists t; +create table t (a int, b int, c int, d int, e int); +insert into t values (1, 2, 3, 4, 5); +alter table t drop column a, drop column d, drop column b; +select * from t; +drop table if exists t; +create table t (a int default 1, c int default 4); +-- error 8200 +alter table t drop column a, drop column a; +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t drop column if exists c, drop column a; +show warnings; +select * from t; +drop table if exists t; +create table t (a int default 1, b int default 2, c int default 3); +insert into t values (); +alter table t drop column a, drop column if exists d, drop column c; +show warnings; +select * from t; + +# TestMultiSchemaChangeAddDropColumns +# [a, b] -> [+c, -a, +d, -b] -> [c, d] +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t add column c int default 3, drop column a, add column d int default 4, drop column b; +select * from t; +# [a, b] -> [-a, -b, +c, +d] -> [c, d] +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t drop column a, drop column b, add column c int default 3, add column d int default 4; +select * from t; +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +-- error 8200 +alter table t add column c int default 3 after a, add column d int default 4 first, drop column a, drop column b; + +# TestMultiSchemaChangeAddIndexes +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t add index t(a, b), add index t1(a); +alter table t add index t2(a), add index t3(a, b); +select * from t use index (t, t1, t2, t3); +admin check table t; +drop table if exists t; +create table t (a int, b int, c int); +-- error 8200 +alter table t add index t(a), add index t(b); +show index from t; +drop table if exists t; +create table t (a int, b int, c int); +-- error 8200 +alter table t add index t(a), drop column a; +-- error 8200 +alter table t add index t(a, b), drop column a; +show index from t; +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 1, 1), (2, 2, 2), (3, 3, 1); +-- error 1062 +alter table t add unique index i1(a), add unique index i2(a, b), add unique index i3(c); +show index from t; +alter table t add index i1(a), add index i2(a, b), add index i3(c); + +# TestMultiSchemaChangeDropIndexes +drop table if exists t; +create table t (a int, b int, c int, index t(a)); +-- error 8200 +alter table t drop index t, drop index t; +drop table if exists t; +create table t (id int, c1 int, c2 int, primary key(id) nonclustered, key i1(c1), key i2(c2), key i3(c1, c2)); +insert into t values (1, 2, 3); +alter table t drop index i1, drop index i2; +-- error 1176 +select * from t use index(i1); +-- error 1176 +select * from t use index(i2); +alter table t drop index i3, drop primary key; +-- error 1176 +select * from t use index(primary); +-- error 1176 +select * from t use index(i3); +drop table if exists t; +create table t (a int default 1, b int default 2, c int default 3, index t(a)); +insert into t values (); +alter table t drop index t, drop column a; +-- error 1176 +select * from t force index(t); + +# TestMultiSchemaChangeAddDropIndexes +drop table if exists t; +create table t (a int, b int, c int, index t(a)); +-- error 1061 +alter table t drop index t, add index t(b); +drop table if exists t; +create table t (a int, b int, c int, index t(a)); +-- error 1091 +alter table t add index t1(b), drop index t1; +drop table if exists t; +create table t (a int, b int, c int, index (a), index(b), index(c)); +insert into t values (1, 2, 3); +alter table t add index xa(a), drop index a, add index xc(c), drop index b, drop index c, add index xb(b); +select * from t use index(xa, xb, xc); +-- error 1176 +select * from t use index(a); +-- error 1176 +select * from t use index(b); +-- error 1176 +select * from t use index(c); +admin check table t; + +# TestMultiSchemaChangeModifyColumns +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +-- error 8200 +alter table t modify column a int default 2, modify column a bigint; +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +-- error 8200 +alter table t modify column b double, drop column b; +drop table if exists t; +create table t (a int default 1, b int default 2, c int default 3); +insert into t values (); +-- error 8200 +alter table t modify column b double after c, drop column c; +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +-- error 8200 +alter table t add index i(a), modify column a int null default 1 after a; +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +-- error 8200 +alter table t add primary key(a), modify column a int null default 1 after a; +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +-- error 8200 +alter table t modify column b double, add index idx((a + b)); +drop table if exists t; +create table t (a int default 1, b int default 2); +insert into t values (); +alter table t modify column b double default 2 after a, add column c int default 3 after a; +select * from t; +drop table if exists t; +create table t (a int, b int, c int); +insert into t values (1, 2, 3); +alter table t modify column a bigint, modify column b bigint; +insert into t values (9223372036854775807, 9223372036854775807, 1); +select * from t; +drop table if exists t; +create table t (a int, b int, c int, index i1(a), index i2(b), index i3(c), index i4(a, b), index i5(a, b, c)); +insert into t values (1, 2, 3); +alter table t modify column a tinyint, modify column b tinyint, modify column c tinyint; +select * from t; +select * from t use index(i1, i2, i3, i4, i5); +admin check table t; +drop table if exists t; +create table t (a int, b int, c int, index i1(a), index i2(b), index i3(c), index i4(a, b), index i5(a, b, c)); +insert into t values (1, 2, 3); +alter table t modify column a tinyint after c, modify column b tinyint, modify column c tinyint first; +select * from t; +select * from t use index(i1, i2, i3, i4, i5); +admin check table t; +drop table if exists t; +create table t (a int, b int, c int, index i1(a), index i2(c, b)); +insert into t values (1, 2, 3), (11, 22, 33); +alter table t modify column b char(255) after c, modify column a bigint; +select * from t; +select * from t use index(i1, i2); +admin check table t; +drop table if exists t; +create table t(a bigint null default '1761233443433596323', index t(a)); +insert into t set a = '-7184819032643664798'; +-- error 1292 +alter table t change column a b datetime null default '8972-12-24 10:56:03', rename index t to t1; +drop table if exists t; +create table t (a int, b double, index i(a, b)); +alter table t rename index i to i1, change column b c int; +select count(*) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='i1'; +drop table if exists t; +create table t (a int, b double, index i(a, b), index _Idx$_i(a, b)); +alter table t rename index i to i1, change column b c int; +select count(*) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='i1'; +select count(*) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t' and COLUMN_NAME='c' and KEY_NAME='_Idx$_i'; +drop table if exists t; +create table t (a int, _Col$_a double, index _Idx$_i(a, _Col$_a), index i(a, _Col$_a)); +alter table t modify column a tinyint; +select count(distinct KEY_NAME) from information_schema.TIDB_INDEXES where TABLE_SCHEMA='ddl__multi_schema_change' and TABLE_NAME='t'; +drop table if exists t; +create table t (a BIGINT NULL DEFAULT '-283977870758975838', b double); +insert into t values (-283977870758975838, 0); +-- error 1690 +alter table t change column a c tinyint null default '111' after b, modify column b time null default '13:51:02' FIRST; +drop table if exists t; +create table t(a int, b int); +insert into t values (1, 2); +-- error 1054 +alter table t add index i(b), modify column a int null default 1 after a; +drop table if exists t; +create table t(a char(3), b int, unique index i1(a), index i2(a, b)); +insert into t values ('aaa', 1), ('aa', 2); +set @@sql_mode = ''; +-- error 1062 +alter table t modify column b tinyint, modify column a char(2); +set @@sql_mode = default; + +# TestMultiSchemaChangeMix +drop table if exists t; +create table t (a int, b int, c int, index i1(c), index i2(c)); +insert into t values (1, 2, 3); +alter table t add column d int default 4, add index i3(c), drop column a, drop column if exists z, add column if not exists e int default 5, drop index i2, add column f int default 6, drop column b, drop index i1, add column if not exists c int; +select * from t; +-- error 1176 +select * from t use index (i1); +-- error 1176 +select * from t use index (i2); +select * from t use index (i3); + +# TestMultiSchemaChangeTableOption +drop table if exists t; +create table t (a int auto_increment primary key, b int) auto_id_cache = 100; +insert into t(b) values(1); +alter table t modify column b tinyint, auto_increment = 200; +insert into t (b) values (2); +select * from t; +drop table if exists t; +create table t (a int auto_increment primary key, b int); +alter table t auto_increment = 110, auto_increment = 90; +show warnings; +insert into t (b) values (1); +select * from t; +drop table if exists t; +create table t (a int, b int) charset = utf8 shard_row_id_bits=2; +alter table t modify column a tinyint, comment = 'abc', charset = utf8mb4; +select TIDB_ROW_ID_SHARDING_INFO, TABLE_COMMENT, TABLE_COLLATION from information_schema.tables where TABLE_SCHEMA='ddl__multi_schema_change' and table_name = 't'; + +# TestMultiSchemaChangeNonPublicDefaultValue +drop table if exists t; +create table t (a tinyint); +insert into t set a = 10; +alter table t add column b int not null, change column a c char(5) first; +select * from t; + +# TestMultiSchemaChangeAlterIndexVisibility +drop table if exists t; +create table t (a int, b int, index idx(b)); +alter table t add index idx2(a), alter index idx visible; +select * from t use index (idx, idx2); +-- error 1176 +alter table t drop column b, alter index idx invisible; +select a, b from t; + +# TestMultiSchemaChangeUnsupportedType +drop table if exists t; +create table t (a int, b int); +-- error 8200 +alter table t add column c int, auto_id_cache = 10; + +# TestMultiSchemaChangeAddIndexChangeColumn +drop table if exists t; +CREATE TABLE t (a SMALLINT DEFAULT '30219', b TIME NULL DEFAULT '02:45:06', PRIMARY KEY (a)); +ALTER TABLE t ADD unique INDEX idx4 (b), change column a e MEDIUMINT DEFAULT '5280454' FIRST; +insert ignore into t (e) values (5586359),(501788),(-5961048),(220083),(-4917129),(-7267211),(7750448); +select * from t; +admin check table t; + +# TestMultiSchemaChangeAddIndexOrder +drop table if exists t; +create table t (a int); +insert into t values (123); +alter table t add index i(a), add primary key (a); +show create table t; +