Skip to content

Commit c24dca5

Browse files
authored
infoschema: load auto id related changes for multi-schema-change diff (#52967)
close #47899
1 parent 7481aa6 commit c24dca5

File tree

7 files changed

+160
-15
lines changed

7 files changed

+160
-15
lines changed

pkg/ddl/multi_schema_change.go

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -127,6 +127,7 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve
127127
subJobs := make([]model.SubJob, len(job.MultiSchemaInfo.SubJobs))
128128
// Step the sub-jobs to the non-revertible states all at once.
129129
// We only generate 1 schema version for these sub-job.
130+
actionTypes := make([]model.ActionType, 0, len(job.MultiSchemaInfo.SubJobs))
130131
for i, sub := range job.MultiSchemaInfo.SubJobs {
131132
if sub.IsFinished() {
132133
continue
@@ -144,13 +145,36 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve
144145
sub.FromProxyJob(&proxyJob, proxyJobVer)
145146
if err != nil || proxyJob.Error != nil {
146147
for j := i - 1; j >= 0; j-- {
148+
// TODO if some sub-job is finished, this will empty them
149+
// also some sub-job cannot be rollback completely, maybe keep them?
147150
job.MultiSchemaInfo.SubJobs[j] = &subJobs[j]
148151
}
149152
handleRevertibleException(job, sub, proxyJob.Error)
150153
// The TableInfo and sub-jobs should be restored
151154
// because some schema changes update the transaction aggressively.
155+
// TODO this error handling cannot handle below case:
156+
// suppose the job is for "alter table t auto_increment = 100, add column c int".
157+
// if we fail on "add column c int", the allocator is rebased to 100
158+
// which cannot be rollback, but it's table-info.AutoIncID is rollback by below call.
159+
// TODO we should also change schema diff of 'ver' if len(actionTypes) > 1.
152160
return updateVersionAndTableInfo(d, t, job, tblInfo, true)
153161
}
162+
actionTypes = append(actionTypes, sub.Type)
163+
}
164+
if len(actionTypes) > 1 {
165+
// only single table schema changes can be put into a multi-schema-change
166+
// job except AddForeignKey which is handled separately in the first loop.
167+
// so this diff is enough, but it wound be better to accumulate all the diffs,
168+
// and then merge them into a single diff.
169+
if err = t.SetSchemaDiff(&model.SchemaDiff{
170+
Version: ver,
171+
Type: job.Type,
172+
TableID: job.TableID,
173+
SchemaID: job.SchemaID,
174+
SubActionTypes: actionTypes,
175+
}); err != nil {
176+
return ver, err
177+
}
154178
}
155179
// All the sub-jobs are non-revertible.
156180
job.MarkNonRevertible()

pkg/infoschema/BUILD.bazel

Lines changed: 2 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -77,6 +77,7 @@ go_test(
7777
timeout = "short",
7878
srcs = [
7979
"bench_test.go",
80+
"builder_test.go",
8081
"infoschema_test.go",
8182
"infoschema_v2_test.go",
8283
"main_test.go",
@@ -85,7 +86,7 @@ go_test(
8586
],
8687
embed = [":infoschema"],
8788
flaky = True,
88-
shard_count = 19,
89+
shard_count = 20,
8990
deps = [
9091
"//pkg/ddl/placement",
9192
"//pkg/domain",

pkg/infoschema/builder.go

Lines changed: 24 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -369,7 +369,7 @@ func (b *Builder) updateBundleForTableUpdate(diff *model.SchemaDiff, newTableID,
369369

370370
func dropTableForUpdate(b *Builder, newTableID, oldTableID int64, dbInfo *model.DBInfo, diff *model.SchemaDiff) ([]int64, autoid.Allocators, error) {
371371
tblIDs := make([]int64, 0, 2)
372-
var newAllocs autoid.Allocators
372+
var keptAllocs autoid.Allocators
373373
// We try to reuse the old allocator, so the cached auto ID can be reused.
374374
if tableIDIsValid(oldTableID) {
375375
if oldTableID == newTableID &&
@@ -386,14 +386,14 @@ func dropTableForUpdate(b *Builder, newTableID, oldTableID int64, dbInfo *model.
386386
// which may have AutoID not connected to tableID
387387
// TODO: can there be _tidb_rowid AutoID per partition?
388388
oldAllocs, _ := allocByID(b, oldTableID)
389-
newAllocs = filterAllocators(diff, oldAllocs)
389+
keptAllocs = getKeptAllocators(diff, oldAllocs)
390390
}
391391

392392
tmpIDs := tblIDs
393393
if (diff.Type == model.ActionRenameTable || diff.Type == model.ActionRenameTables) && diff.OldSchemaID != diff.SchemaID {
394394
oldDBInfo, ok := oldSchemaInfo(b, diff)
395395
if !ok {
396-
return nil, newAllocs, ErrDatabaseNotExists.GenWithStackByArgs(
396+
return nil, keptAllocs, ErrDatabaseNotExists.GenWithStackByArgs(
397397
fmt.Sprintf("(Schema ID %d)", diff.OldSchemaID),
398398
)
399399
}
@@ -407,7 +407,7 @@ func dropTableForUpdate(b *Builder, newTableID, oldTableID int64, dbInfo *model.
407407
tblIDs = tmpIDs
408408
}
409409
}
410-
return tblIDs, newAllocs, nil
410+
return tblIDs, keptAllocs, nil
411411
}
412412

413413
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
438438
return tblIDs, nil
439439
}
440440

441-
func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators {
442-
var newAllocs autoid.Allocators
441+
// getKeptAllocators get allocators that is not changed by the DDL.
442+
func getKeptAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators {
443+
var autoIDChanged, autoRandomChanged bool
443444
switch diff.Type {
444445
case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache:
446+
autoIDChanged = true
447+
case model.ActionRebaseAutoRandomBase:
448+
autoRandomChanged = true
449+
case model.ActionMultiSchemaChange:
450+
for _, t := range diff.SubActionTypes {
451+
switch t {
452+
case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache:
453+
autoIDChanged = true
454+
case model.ActionRebaseAutoRandomBase:
455+
autoRandomChanged = true
456+
}
457+
}
458+
}
459+
var newAllocs autoid.Allocators
460+
switch {
461+
case autoIDChanged:
445462
// Only drop auto-increment allocator.
446463
newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool {
447464
tp := a.GetType()
448465
return tp != autoid.RowIDAllocType && tp != autoid.AutoIncrementType
449466
})
450-
case model.ActionRebaseAutoRandomBase:
467+
case autoRandomChanged:
451468
// Only drop auto-random allocator.
452469
newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool {
453470
tp := a.GetType()

pkg/infoschema/builder_test.go

Lines changed: 94 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,94 @@
1+
// Copyright 2024 PingCAP, Inc.
2+
//
3+
// Licensed under the Apache License, Version 2.0 (the "License");
4+
// you may not use this file except in compliance with the License.
5+
// You may obtain a copy of the License at
6+
//
7+
// http://www.apache.org/licenses/LICENSE-2.0
8+
//
9+
// Unless required by applicable law or agreed to in writing, software
10+
// distributed under the License is distributed on an "AS IS" BASIS,
11+
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
12+
// See the License for the specific language governing permissions and
13+
// limitations under the License.
14+
15+
package infoschema
16+
17+
import (
18+
"fmt"
19+
"testing"
20+
21+
"github.com/pingcap/tidb/pkg/meta/autoid"
22+
"github.com/pingcap/tidb/pkg/parser/model"
23+
"github.com/stretchr/testify/require"
24+
)
25+
26+
type mockAlloc struct {
27+
autoid.Allocator
28+
tp autoid.AllocatorType
29+
}
30+
31+
func (m *mockAlloc) GetType() autoid.AllocatorType {
32+
return m.tp
33+
}
34+
35+
func TestGetKeptAllocators(t *testing.T) {
36+
checkAllocators := func(allocators autoid.Allocators, expected []autoid.AllocatorType) {
37+
require.Len(t, allocators.Allocs, len(expected))
38+
for i, tp := range expected {
39+
require.Equal(t, tp, allocators.Allocs[i].GetType())
40+
}
41+
}
42+
allocators := autoid.Allocators{Allocs: []autoid.Allocator{
43+
&mockAlloc{tp: autoid.RowIDAllocType},
44+
&mockAlloc{tp: autoid.AutoIncrementType},
45+
&mockAlloc{tp: autoid.AutoRandomType},
46+
}}
47+
cases := []struct {
48+
diff *model.SchemaDiff
49+
expected []autoid.AllocatorType
50+
}{
51+
{
52+
diff: &model.SchemaDiff{Type: model.ActionTruncateTable},
53+
expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType, autoid.AutoRandomType},
54+
},
55+
{
56+
diff: &model.SchemaDiff{Type: model.ActionRebaseAutoID},
57+
expected: []autoid.AllocatorType{autoid.AutoRandomType},
58+
},
59+
{
60+
diff: &model.SchemaDiff{Type: model.ActionModifyTableAutoIdCache},
61+
expected: []autoid.AllocatorType{autoid.AutoRandomType},
62+
},
63+
{
64+
diff: &model.SchemaDiff{Type: model.ActionRebaseAutoRandomBase},
65+
expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType},
66+
},
67+
{
68+
diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange,
69+
SubActionTypes: []model.ActionType{model.ActionAddColumn, model.ActionRebaseAutoID}},
70+
expected: []autoid.AllocatorType{autoid.AutoRandomType},
71+
},
72+
{
73+
diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange,
74+
SubActionTypes: []model.ActionType{model.ActionModifyTableAutoIdCache}},
75+
expected: []autoid.AllocatorType{autoid.AutoRandomType},
76+
},
77+
{
78+
diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange,
79+
SubActionTypes: []model.ActionType{model.ActionRebaseAutoRandomBase}},
80+
expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType},
81+
},
82+
{
83+
diff: &model.SchemaDiff{Type: model.ActionMultiSchemaChange,
84+
SubActionTypes: []model.ActionType{model.ActionAddColumn}},
85+
expected: []autoid.AllocatorType{autoid.RowIDAllocType, autoid.AutoIncrementType, autoid.AutoRandomType},
86+
},
87+
}
88+
for i, c := range cases {
89+
t.Run(fmt.Sprintf("case %d", i), func(t *testing.T) {
90+
res := getKeptAllocators(c.diff, allocators)
91+
checkAllocators(res, c.expected)
92+
})
93+
}
94+
}

pkg/parser/model/ddl.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1138,6 +1138,12 @@ type SchemaDiff struct {
11381138
SchemaID int64 `json:"schema_id"`
11391139
TableID int64 `json:"table_id"`
11401140

1141+
// SubActionTypes is the list of action types done together within a multiple schema
1142+
// change job. As the job might contain multiple steps that changes schema version,
1143+
// if some step only contains one action, Type will be that action, and SubActionTypes
1144+
// will be empty.
1145+
// for other types of job, it will always be empty.
1146+
SubActionTypes []ActionType `json:"sub_action_types,omitempty"`
11411147
// OldTableID is the table ID before truncate, only used by truncate table DDL.
11421148
OldTableID int64 `json:"old_table_id"`
11431149
// OldSchemaID is the schema ID before rename table, only used by rename table DDL.

tests/integrationtest/r/ddl/multi_schema_change.result

Lines changed: 6 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -373,12 +373,14 @@ select * from t use index (i3);
373373
c d e f
374374
3 4 5 6
375375
drop table if exists t;
376-
create table t (a int auto_increment primary key, b int);
377-
alter table t modify column b tinyint, auto_increment = 100;
378-
insert into t (b) values (1);
376+
create table t (a int auto_increment primary key, b int) auto_id_cache = 100;
377+
insert into t(b) values(1);
378+
alter table t modify column b tinyint, auto_increment = 200;
379+
insert into t (b) values (2);
379380
select * from t;
380381
a b
381-
100 1
382+
1 1
383+
200 2
382384
drop table if exists t;
383385
create table t (a int auto_increment primary key, b int);
384386
alter table t auto_increment = 110, auto_increment = 90;

tests/integrationtest/t/ddl/multi_schema_change.test

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -312,9 +312,10 @@ select * from t use index (i3);
312312

313313
# TestMultiSchemaChangeTableOption
314314
drop table if exists t;
315-
create table t (a int auto_increment primary key, b int);
316-
alter table t modify column b tinyint, auto_increment = 100;
317-
insert into t (b) values (1);
315+
create table t (a int auto_increment primary key, b int) auto_id_cache = 100;
316+
insert into t(b) values(1);
317+
alter table t modify column b tinyint, auto_increment = 200;
318+
insert into t (b) values (2);
318319
select * from t;
319320
drop table if exists t;
320321
create table t (a int auto_increment primary key, b int);

0 commit comments

Comments
 (0)