Skip to content

Commit 6ae3363

Browse files
D3Hunterti-chi-bot
authored andcommitted
This is an automated cherry-pick of pingcap#52967
Signed-off-by: ti-chi-bot <[email protected]>
1 parent cc04dd7 commit 6ae3363

File tree

7 files changed

+1104
-3
lines changed

7 files changed

+1104
-3
lines changed

ddl/multi_schema_change.go

Lines changed: 24 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -110,6 +110,7 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve
110110
subJobs := make([]model.SubJob, len(job.MultiSchemaInfo.SubJobs))
111111
// Step the sub-jobs to the non-revertible states all at once.
112112
// We only generate 1 schema version for these sub-job.
113+
actionTypes := make([]model.ActionType, 0, len(job.MultiSchemaInfo.SubJobs))
113114
for i, sub := range job.MultiSchemaInfo.SubJobs {
114115
if sub.IsFinished() {
115116
continue
@@ -127,13 +128,36 @@ func onMultiSchemaChange(w *worker, d *ddlCtx, t *meta.Meta, job *model.Job) (ve
127128
sub.FromProxyJob(&proxyJob, proxyJobVer)
128129
if err != nil || proxyJob.Error != nil {
129130
for j := i - 1; j >= 0; j-- {
131+
// TODO if some sub-job is finished, this will empty them
132+
// also some sub-job cannot be rollback completely, maybe keep them?
130133
job.MultiSchemaInfo.SubJobs[j] = &subJobs[j]
131134
}
132135
handleRevertibleException(job, sub, proxyJob.Error)
133136
// The TableInfo and sub-jobs should be restored
134137
// because some schema changes update the transaction aggressively.
138+
// TODO this error handling cannot handle below case:
139+
// suppose the job is for "alter table t auto_increment = 100, add column c int".
140+
// if we fail on "add column c int", the allocator is rebased to 100
141+
// which cannot be rollback, but it's table-info.AutoIncID is rollback by below call.
142+
// TODO we should also change schema diff of 'ver' if len(actionTypes) > 1.
135143
return updateVersionAndTableInfo(d, t, job, tblInfo, true)
136144
}
145+
actionTypes = append(actionTypes, sub.Type)
146+
}
147+
if len(actionTypes) > 1 {
148+
// only single table schema changes can be put into a multi-schema-change
149+
// job except AddForeignKey which is handled separately in the first loop.
150+
// so this diff is enough, but it wound be better to accumulate all the diffs,
151+
// and then merge them into a single diff.
152+
if err = t.SetSchemaDiff(&model.SchemaDiff{
153+
Version: ver,
154+
Type: job.Type,
155+
TableID: job.TableID,
156+
SchemaID: job.SchemaID,
157+
SubActionTypes: actionTypes,
158+
}); err != nil {
159+
return ver, err
160+
}
137161
}
138162
// All the sub-jobs are non-revertible.
139163
job.MarkNonRevertible()

infoschema/builder.go

Lines changed: 59 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -488,6 +488,10 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6
488488
b.copySortedTables(oldTableID, newTableID)
489489

490490
tblIDs := make([]int64, 0, 2)
491+
<<<<<<< HEAD:infoschema/builder.go
492+
=======
493+
var keptAllocs autoid.Allocators
494+
>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go
491495
// We try to reuse the old allocator, so the cached auto ID can be reused.
492496
var allocs autoid.Allocators
493497
if tableIDIsValid(oldTableID) {
@@ -499,15 +503,27 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6
499503
diff.Type != model.ActionRepairTable &&
500504
// Alter sequence will change the sequence info in the allocator, so the old allocator is not valid any more.
501505
diff.Type != model.ActionAlterSequence {
506+
<<<<<<< HEAD:infoschema/builder.go
502507
oldAllocs, _ := b.is.AllocByID(oldTableID)
503508
allocs = filterAllocators(diff, oldAllocs)
509+
=======
510+
// TODO: Check how this would work with ADD/REMOVE Partitioning,
511+
// which may have AutoID not connected to tableID
512+
// TODO: can there be _tidb_rowid AutoID per partition?
513+
oldAllocs, _ := allocByID(b, oldTableID)
514+
keptAllocs = getKeptAllocators(diff, oldAllocs)
515+
>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go
504516
}
505517

506518
tmpIDs := tblIDs
507519
if (diff.Type == model.ActionRenameTable || diff.Type == model.ActionRenameTables) && diff.OldSchemaID != diff.SchemaID {
508520
oldRoDBInfo, ok := b.is.SchemaByID(diff.OldSchemaID)
509521
if !ok {
522+
<<<<<<< HEAD:infoschema/builder.go
510523
return nil, ErrDatabaseNotExists.GenWithStackByArgs(
524+
=======
525+
return nil, keptAllocs, ErrDatabaseNotExists.GenWithStackByArgs(
526+
>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go
511527
fmt.Sprintf("(Schema ID %d)", diff.OldSchemaID),
512528
)
513529
}
@@ -522,6 +538,29 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6
522538
tblIDs = tmpIDs
523539
}
524540
}
541+
<<<<<<< HEAD:infoschema/builder.go
542+
=======
543+
return tblIDs, keptAllocs, nil
544+
}
545+
546+
func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int64, error) {
547+
roDBInfo, ok := b.infoSchema.SchemaByID(diff.SchemaID)
548+
if !ok {
549+
return nil, ErrDatabaseNotExists.GenWithStackByArgs(
550+
fmt.Sprintf("(Schema ID %d)", diff.SchemaID),
551+
)
552+
}
553+
dbInfo := b.getSchemaAndCopyIfNecessary(roDBInfo.Name.L)
554+
oldTableID, newTableID := b.getTableIDs(diff)
555+
b.updateBundleForTableUpdate(diff, newTableID, oldTableID)
556+
b.copySortedTables(oldTableID, newTableID)
557+
558+
tblIDs, allocs, err := dropTableForUpdate(b, newTableID, oldTableID, dbInfo, diff)
559+
if err != nil {
560+
return nil, err
561+
}
562+
563+
>>>>>>> c24dca58f5e (infoschema: load auto id related changes for multi-schema-change diff (#52967)):pkg/infoschema/builder.go
525564
if tableIDIsValid(newTableID) {
526565
// All types except DropTableOrView.
527566
var err error
@@ -533,16 +572,33 @@ func (b *Builder) applyTableUpdate(m *meta.Meta, diff *model.SchemaDiff) ([]int6
533572
return tblIDs, nil
534573
}
535574

536-
func filterAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators {
537-
var newAllocs autoid.Allocators
575+
// getKeptAllocators get allocators that is not changed by the DDL.
576+
func getKeptAllocators(diff *model.SchemaDiff, oldAllocs autoid.Allocators) autoid.Allocators {
577+
var autoIDChanged, autoRandomChanged bool
538578
switch diff.Type {
539579
case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache:
580+
autoIDChanged = true
581+
case model.ActionRebaseAutoRandomBase:
582+
autoRandomChanged = true
583+
case model.ActionMultiSchemaChange:
584+
for _, t := range diff.SubActionTypes {
585+
switch t {
586+
case model.ActionRebaseAutoID, model.ActionModifyTableAutoIdCache:
587+
autoIDChanged = true
588+
case model.ActionRebaseAutoRandomBase:
589+
autoRandomChanged = true
590+
}
591+
}
592+
}
593+
var newAllocs autoid.Allocators
594+
switch {
595+
case autoIDChanged:
540596
// Only drop auto-increment allocator.
541597
newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool {
542598
tp := a.GetType()
543599
return tp != autoid.RowIDAllocType && tp != autoid.AutoIncrementType
544600
})
545-
case model.ActionRebaseAutoRandomBase:
601+
case autoRandomChanged:
546602
// Only drop auto-random allocator.
547603
newAllocs = oldAllocs.Filter(func(a autoid.Allocator) bool {
548604
tp := a.GetType()

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+
}

parser/model/ddl.go

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -998,6 +998,12 @@ type SchemaDiff struct {
998998
SchemaID int64 `json:"schema_id"`
999999
TableID int64 `json:"table_id"`
10001000

1001+
// SubActionTypes is the list of action types done together within a multiple schema
1002+
// change job. As the job might contain multiple steps that changes schema version,
1003+
// if some step only contains one action, Type will be that action, and SubActionTypes
1004+
// will be empty.
1005+
// for other types of job, it will always be empty.
1006+
SubActionTypes []ActionType `json:"sub_action_types,omitempty"`
10011007
// OldTableID is the table ID before truncate, only used by truncate table DDL.
10021008
OldTableID int64 `json:"old_table_id"`
10031009
// OldSchemaID is the schema ID before rename table, only used by rename table DDL.

pkg/infoschema/BUILD.bazel

Lines changed: 116 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,116 @@
1+
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
2+
3+
go_library(
4+
name = "infoschema",
5+
srcs = [
6+
"builder.go",
7+
"builder_misc.go",
8+
"bundle_builder.go",
9+
"cache.go",
10+
"cluster.go",
11+
"error.go",
12+
"infoschema.go",
13+
"infoschema_v2.go",
14+
"interface.go",
15+
"metric_table_def.go",
16+
"metrics_schema.go",
17+
"sieve.go",
18+
"tables.go",
19+
],
20+
importpath = "github.com/pingcap/tidb/pkg/infoschema",
21+
visibility = ["//visibility:public"],
22+
deps = [
23+
"//pkg/config",
24+
"//pkg/ddl/placement",
25+
"//pkg/ddl/resourcegroup",
26+
"//pkg/domain/infosync",
27+
"//pkg/errno",
28+
"//pkg/infoschema/context",
29+
"//pkg/infoschema/internal",
30+
"//pkg/infoschema/metrics",
31+
"//pkg/kv",
32+
"//pkg/meta",
33+
"//pkg/meta/autoid",
34+
"//pkg/parser/auth",
35+
"//pkg/parser/charset",
36+
"//pkg/parser/model",
37+
"//pkg/parser/mysql",
38+
"//pkg/parser/terror",
39+
"//pkg/privilege",
40+
"//pkg/session/txninfo",
41+
"//pkg/sessionctx",
42+
"//pkg/sessionctx/variable",
43+
"//pkg/table",
44+
"//pkg/table/tables",
45+
"//pkg/types",
46+
"//pkg/util",
47+
"//pkg/util/dbterror",
48+
"//pkg/util/deadlockhistory",
49+
"//pkg/util/domainutil",
50+
"//pkg/util/execdetails",
51+
"//pkg/util/intest",
52+
"//pkg/util/logutil",
53+
"//pkg/util/mock",
54+
"//pkg/util/sem",
55+
"//pkg/util/set",
56+
"//pkg/util/size",
57+
"//pkg/util/stmtsummary",
58+
"@com_github_ngaut_pools//:pools",
59+
"@com_github_pingcap_errors//:errors",
60+
"@com_github_pingcap_failpoint//:failpoint",
61+
"@com_github_pingcap_kvproto//pkg/diagnosticspb",
62+
"@com_github_pingcap_kvproto//pkg/metapb",
63+
"@com_github_pingcap_log//:log",
64+
"@com_github_tidwall_btree//:btree",
65+
"@com_github_tikv_client_go_v2//tikv",
66+
"@com_github_tikv_pd_client//http",
67+
"@org_golang_google_grpc//:grpc",
68+
"@org_golang_google_grpc//credentials",
69+
"@org_golang_google_grpc//credentials/insecure",
70+
"@org_golang_x_sync//singleflight",
71+
"@org_uber_go_zap//:zap",
72+
],
73+
)
74+
75+
go_test(
76+
name = "infoschema_test",
77+
timeout = "short",
78+
srcs = [
79+
"bench_test.go",
80+
"builder_test.go",
81+
"infoschema_test.go",
82+
"infoschema_v2_test.go",
83+
"main_test.go",
84+
"metrics_schema_test.go",
85+
"sieve_test.go",
86+
],
87+
embed = [":infoschema"],
88+
flaky = True,
89+
shard_count = 20,
90+
deps = [
91+
"//pkg/ddl/placement",
92+
"//pkg/domain",
93+
"//pkg/infoschema/internal",
94+
"//pkg/kv",
95+
"//pkg/meta",
96+
"//pkg/meta/autoid",
97+
"//pkg/parser/model",
98+
"//pkg/parser/mysql",
99+
"//pkg/sessionctx/variable",
100+
"//pkg/store/driver",
101+
"//pkg/table",
102+
"//pkg/testkit",
103+
"//pkg/testkit/testsetup",
104+
"//pkg/testkit/testutil",
105+
"//pkg/types",
106+
"//pkg/util",
107+
"//pkg/util/logutil",
108+
"//pkg/util/set",
109+
"//pkg/util/size",
110+
"@com_github_pingcap_errors//:errors",
111+
"@com_github_prometheus_prometheus//promql/parser",
112+
"@com_github_stretchr_testify//require",
113+
"@org_uber_go_goleak//:goleak",
114+
"@org_uber_go_zap//:zap",
115+
],
116+
)

0 commit comments

Comments
 (0)