Skip to content

Commit

Permalink
ddl: disable fast reorg and dist task execution for system tables (#4…
Browse files Browse the repository at this point in the history
  • Loading branch information
tangenta authored Dec 18, 2023
1 parent 7fd5460 commit 8ed0d47
Show file tree
Hide file tree
Showing 4 changed files with 25 additions and 7 deletions.
18 changes: 12 additions & 6 deletions pkg/ddl/ddl_api.go
Original file line number Diff line number Diff line change
Expand Up @@ -7092,7 +7092,7 @@ func (d *ddl) CreatePrimaryKey(ctx sessionctx.Context, ti ast.Ident, indexName m
Args: []interface{}{unique, indexName, indexPartSpecifications, indexOption, sqlMode, nil, global},
Priority: ctx.GetSessionVars().DDLReorgPriority,
}
reorgMeta, err := newReorgMetaFromVariables(d, job, ctx)
reorgMeta, err := newReorgMetaFromVariables(job, ctx)
if err != nil {
return err
}
Expand Down Expand Up @@ -7349,7 +7349,7 @@ func (d *ddl) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde
Charset: chs,
Collate: coll,
}
reorgMeta, err := newReorgMetaFromVariables(d, job, ctx)
reorgMeta, err := newReorgMetaFromVariables(job, ctx)
if err != nil {
return err
}
Expand All @@ -7365,24 +7365,30 @@ func (d *ddl) createIndex(ctx sessionctx.Context, ti ast.Ident, keyType ast.Inde
return errors.Trace(err)
}

func newReorgMetaFromVariables(d *ddl, job *model.Job, sctx sessionctx.Context) (*model.DDLReorgMeta, error) {
func newReorgMetaFromVariables(job *model.Job, sctx sessionctx.Context) (*model.DDLReorgMeta, error) {
reorgMeta := NewDDLReorgMeta(sctx)
reorgMeta.IsDistReorg = variable.EnableDistTask.Load()
reorgMeta.IsFastReorg = variable.EnableFastReorg.Load()
if reorgMeta.IsDistReorg && !reorgMeta.IsFastReorg {
return nil, dbterror.ErrUnsupportedDistTask
}
isUpgradingSysDB := d.stateSyncer.IsUpgradingState() && hasSysDB(job)
if isUpgradingSysDB {
if hasSysDB(job) {
if reorgMeta.IsDistReorg {
logutil.BgLogger().Info("cannot use distributed task execution because the job on system DB is in upgrade state",
logutil.BgLogger().Info("cannot use distributed task execution on system DB",
zap.String("category", "ddl"), zap.Stringer("job", job))
}
reorgMeta.IsDistReorg = false
reorgMeta.IsFastReorg = false
failpoint.Inject("reorgMetaRecordFastReorgDisabled", func(_ failpoint.Value) {
LastReorgMetaFastReorgDisabled = true
})
}
return reorgMeta, nil
}

// LastReorgMetaFastReorgDisabled is used for test.
var LastReorgMetaFastReorgDisabled bool

func buildFKInfo(fkName model.CIStr, keys []*ast.IndexPartSpecification, refer *ast.ReferenceDef, cols []*table.Column) (*model.FKInfo, error) {
if len(keys) != len(refer.IndexPartSpecifications) {
return nil, infoschema.ErrForeignKeyNotMatch.GenWithStackByArgs(fkName, "Key reference and table reference don't match")
Expand Down
2 changes: 1 addition & 1 deletion pkg/ddl/multi_schema_change.go
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ func (d *ddl) MultiSchemaChange(ctx sessionctx.Context, ti ast.Ident) error {
ReorgMeta: nil,
}
if containsDistTaskSubJob(subJobs) {
job.ReorgMeta, err = newReorgMetaFromVariables(d, job, ctx)
job.ReorgMeta, err = newReorgMetaFromVariables(job, ctx)
if err != nil {
return err
}
Expand Down
2 changes: 2 additions & 0 deletions pkg/session/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -136,6 +136,7 @@ go_test(
"//pkg/autoid_service",
"//pkg/bindinfo",
"//pkg/config",
"//pkg/ddl",
"//pkg/domain",
"//pkg/executor",
"//pkg/expression",
Expand All @@ -157,6 +158,7 @@ go_test(
"//pkg/util/chunk",
"//pkg/util/logutil",
"//pkg/util/sqlexec",
"@com_github_pingcap_failpoint//:failpoint",
"@com_github_pingcap_log//:log",
"@com_github_stretchr_testify//require",
"@com_github_tikv_client_go_v2//tikv",
Expand Down
10 changes: 10 additions & 0 deletions pkg/session/bootstrap_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,7 +21,9 @@ import (
"testing"
"time"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/bindinfo"
"github.com/pingcap/tidb/pkg/ddl"
"github.com/pingcap/tidb/pkg/domain"
"github.com/pingcap/tidb/pkg/meta"
"github.com/pingcap/tidb/pkg/parser/auth"
Expand Down Expand Up @@ -1585,10 +1587,18 @@ func TestTiDBUpgradeToVer136(t *testing.T) {
require.NoError(t, err)
require.Equal(t, int64(ver135), ver)

MustExec(t, seV135, "ALTER TABLE mysql.tidb_background_subtask DROP INDEX idx_task_key;")
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/ddl/reorgMetaRecordFastReorgDisabled", `return`))
t.Cleanup(func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/ddl/reorgMetaRecordFastReorgDisabled"))
})
MustExec(t, seV135, "set global tidb_ddl_enable_fast_reorg = 1")
dom, err := BootstrapSession(store)
require.NoError(t, err)
ver, err = getBootstrapVersion(seV135)
require.NoError(t, err)
require.True(t, ddl.LastReorgMetaFastReorgDisabled)

require.Less(t, int64(ver135), ver)
dom.Close()
}
Expand Down

0 comments on commit 8ed0d47

Please sign in to comment.