Skip to content

Commit

Permalink
infoschema: load auto id related changes for multi-schema-change diff (
Browse files Browse the repository at this point in the history
  • Loading branch information
D3Hunter committed Apr 30, 2024
1 parent 7481aa6 commit c24dca5
Show file tree
Hide file tree
Showing 7 changed files with 160 additions and 15 deletions.
24 changes: 24 additions & 0 deletions pkg/ddl/multi_schema_change.go
Original file line number Diff line number Diff line change
Expand Up @@ -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
Expand All @@ -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()
Expand Down
3 changes: 2 additions & 1 deletion pkg/infoschema/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ go_test(
timeout = "short",
srcs = [
"bench_test.go",
"builder_test.go",
"infoschema_test.go",
"infoschema_v2_test.go",
"main_test.go",
Expand All @@ -85,7 +86,7 @@ go_test(
],
embed = [":infoschema"],
flaky = True,
shard_count = 19,
shard_count = 20,
deps = [
"//pkg/ddl/placement",
"//pkg/domain",
Expand Down
31 changes: 24 additions & 7 deletions pkg/infoschema/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 &&
Expand All @@ -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),
)
}
Expand All @@ -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) {
Expand Down Expand Up @@ -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()
Expand Down
94 changes: 94 additions & 0 deletions pkg/infoschema/builder_test.go
Original file line number Diff line number Diff line change
@@ -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)
})
}
}
6 changes: 6 additions & 0 deletions pkg/parser/model/ddl.go
Original file line number Diff line number Diff line change
Expand Up @@ -1138,6 +1138,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.
Expand Down
10 changes: 6 additions & 4 deletions tests/integrationtest/r/ddl/multi_schema_change.result
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down
7 changes: 4 additions & 3 deletions tests/integrationtest/t/ddl/multi_schema_change.test
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down

0 comments on commit c24dca5

Please sign in to comment.