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/BUILD.bazel b/infoschema/BUILD.bazel index 93f059dcd3e61..4430cfbc552ff 100644 --- a/infoschema/BUILD.bazel +++ b/infoschema/BUILD.bazel @@ -68,6 +68,7 @@ go_test( name = "infoschema_test", timeout = "short", srcs = [ + "builder_test.go", "cache_test.go", "cluster_tables_test.go", "infoschema_test.go", diff --git a/infoschema/builder.go b/infoschema/builder.go index 792007feb82e6..24b217231a5fa 100644 --- a/infoschema/builder.go +++ b/infoschema/builder.go @@ -489,8 +489,8 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 b.copySortedTables(oldTableID, newTableID) tblIDs := make([]int64, 0, 2) + var keptAllocs autoid.Allocators // We try to reuse the old allocator, so the cached auto ID can be reused. - var allocs autoid.Allocators if tableIDIsValid(oldTableID) { if oldTableID == newTableID && (diff.Type != model.ActionRenameTable && diff.Type != model.ActionRenameTables) && // For repairing table in TiDB cluster, given 2 normal node and 1 repair node. @@ -500,8 +500,11 @@ 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) + keptAllocs = getKeptAllocators(diff, oldAllocs) } tmpIDs := tblIDs @@ -526,7 +529,7 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6 if tableIDIsValid(newTableID) { // All types except DropTableOrView. var err error - tblIDs, err = b.applyCreateTable(m, dbInfo, newTableID, allocs, diff.Type, tblIDs) + tblIDs, err = b.applyCreateTable(m, dbInfo, newTableID, keptAllocs, diff.Type, tblIDs) if err != nil { return nil, errors.Trace(err) } @@ -534,16 +537,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..117d923f34f3c --- /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/meta/autoid" + "github.com/pingcap/tidb/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/tests/integrationtest/r/ddl/multi_schema_change.result b/tests/integrationtest/r/ddl/multi_schema_change.result new file mode 100644 index 0000000000000..d7ffba91144a6 --- /dev/null +++ b/tests/integrationtest/r/ddl/multi_schema_change.result @@ -0,0 +1,9 @@ +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 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..ce96326cb92cd --- /dev/null +++ b/tests/integrationtest/t/ddl/multi_schema_change.test @@ -0,0 +1,7 @@ +# 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;