diff --git a/pkg/ddl/multi_schema_change.go b/pkg/ddl/multi_schema_change.go index 7d9c477f7b4e5..a9291f79124e1 100644 --- a/pkg/ddl/multi_schema_change.go +++ b/pkg/ddl/multi_schema_change.go @@ -127,6 +127,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 @@ -144,13 +145,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/pkg/infoschema/BUILD.bazel b/pkg/infoschema/BUILD.bazel index 8413c1682919b..201de5ed3f294 100644 --- a/pkg/infoschema/BUILD.bazel +++ b/pkg/infoschema/BUILD.bazel @@ -76,6 +76,7 @@ go_test( timeout = "short", srcs = [ "bench_test.go", + "builder_test.go", "infoschema_test.go", "infoschema_v2_test.go", "main_test.go", @@ -83,7 +84,7 @@ go_test( ], embed = [":infoschema"], flaky = True, - shard_count = 13, + shard_count = 14, deps = [ "//pkg/ddl/placement", "//pkg/domain", diff --git a/pkg/infoschema/builder.go b/pkg/infoschema/builder.go index 6b9b529972d6e..d03b07d7d1afa 100644 --- a/pkg/infoschema/builder.go +++ b/pkg/infoschema/builder.go @@ -369,7 +369,7 @@ func (b *Builder) updateBundleForTableUpdate(diff *model.SchemaDiff, newTableID, func dropTableForUpdate(b *Builder, newTableID, oldTableID int64, dbInfo *model.DBInfo, diff *model.SchemaDiff) ([]int64, autoid.Allocators, error) { tblIDs := make([]int64, 0, 2) - var newAllocs autoid.Allocators + var keptAllocs autoid.Allocators // We try to reuse the old allocator, so the cached auto ID can be reused. if tableIDIsValid(oldTableID) { if oldTableID == newTableID && @@ -386,14 +386,14 @@ func dropTableForUpdate(b *Builder, newTableID, oldTableID int64, dbInfo *model. // which may have AutoID not connected to tableID // TODO: can there be _tidb_rowid AutoID per partition? oldAllocs, _ := allocByID(b, oldTableID) - newAllocs = filterAllocators(diff, oldAllocs) + keptAllocs = getKeptAllocators(diff, oldAllocs) } tmpIDs := tblIDs if (diff.Type == model.ActionRenameTable || diff.Type == model.ActionRenameTables) && diff.OldSchemaID != diff.SchemaID { oldDBInfo, ok := oldSchemaInfo(b, diff) if !ok { - return nil, newAllocs, ErrDatabaseNotExists.GenWithStackByArgs( + return nil, keptAllocs, ErrDatabaseNotExists.GenWithStackByArgs( fmt.Sprintf("(Schema ID %d)", diff.OldSchemaID), ) } @@ -407,7 +407,7 @@ func dropTableForUpdate(b *Builder, newTableID, oldTableID int64, dbInfo *model. tblIDs = tmpIDs } } - return tblIDs, newAllocs, nil + return tblIDs, keptAllocs, nil } func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int64, error) { @@ -438,16 +438,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/pkg/infoschema/builder_test.go b/pkg/infoschema/builder_test.go new file mode 100644 index 0000000000000..cd9d34c45bcfd --- /dev/null +++ b/pkg/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/pkg/parser/model/ddl.go b/pkg/parser/model/ddl.go index df525f8dcbd47..cae4ed84d2d4d 100644 --- a/pkg/parser/model/ddl.go +++ b/pkg/parser/model/ddl.go @@ -1126,6 +1126,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/tests/integrationtest/r/ddl/multi_schema_change.result b/tests/integrationtest/r/ddl/multi_schema_change.result index c69439a79a459..d363a29075937 100644 --- a/tests/integrationtest/r/ddl/multi_schema_change.result +++ b/tests/integrationtest/r/ddl/multi_schema_change.result @@ -373,12 +373,14 @@ 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); -alter table t modify column b tinyint, auto_increment = 100; -insert into t (b) values (1); +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 -100 1 +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; diff --git a/tests/integrationtest/t/ddl/multi_schema_change.test b/tests/integrationtest/t/ddl/multi_schema_change.test index 29e8a6fe2403c..3fc412745096d 100644 --- a/tests/integrationtest/t/ddl/multi_schema_change.test +++ b/tests/integrationtest/t/ddl/multi_schema_change.test @@ -312,9 +312,10 @@ select * from t use index (i3); # TestMultiSchemaChangeTableOption drop table if exists t; -create table t (a int auto_increment primary key, b int); -alter table t modify column b tinyint, auto_increment = 100; -insert into t (b) values (1); +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);