Skip to content

Commit

Permalink
ddl: create temp dir automatically for adding index (#45457)
Browse files Browse the repository at this point in the history
close #45456
  • Loading branch information
tangenta committed Jul 20, 2023
1 parent c6d83d7 commit 7f016ce
Show file tree
Hide file tree
Showing 7 changed files with 44 additions and 6 deletions.
5 changes: 4 additions & 1 deletion ddl/index.go
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,10 @@ func (w *worker) onCreateIndex(d *ddlCtx, t *meta.Meta, job *model.Job, isPK boo
var reorgTp model.ReorgType
reorgTp, err = pickBackfillType(w.ctx, job, indexInfo.Unique, d)
if err != nil {
break
if !errorIsRetryable(err, job) {
job.State = model.JobStateCancelled
}
return ver, err
}
if reorgTp.NeedMergeProcess() {
// Increase telemetryAddIndexIngestUsage
Expand Down
15 changes: 13 additions & 2 deletions ddl/ingest/disk_root.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,12 +16,15 @@ package ingest

import (
"fmt"
"os"
"sync"
"sync/atomic"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
lcom "github.com/pingcap/tidb/br/pkg/lightning/common"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/dbterror"
"github.com/pingcap/tidb/util/logutil"
"go.uber.org/zap"
)
Expand Down Expand Up @@ -114,13 +117,21 @@ func (d *diskRootImpl) usageInfo() string {

// PreCheckUsage implements DiskRoot interface.
func (d *diskRootImpl) PreCheckUsage() error {
failpoint.Inject("mockIngestCheckEnvFailed", func(_ failpoint.Value) {
failpoint.Return(dbterror.ErrIngestCheckEnvFailed.FastGenByArgs("mock error"))
})
err := os.MkdirAll(d.path, 0700)
if err != nil {
return dbterror.ErrIngestCheckEnvFailed.FastGenByArgs(err.Error())
}
sz, err := lcom.GetStorageSize(d.path)
if err != nil {
return errors.Trace(err)
return dbterror.ErrIngestCheckEnvFailed.FastGenByArgs(err.Error())
}
if RiskOfDiskFull(sz.Available, sz.Capacity) {
sortPath := ConfigSortPath()
return errors.Errorf("sort path: %s, %s, please clean up the disk and retry", sortPath, d.UsageInfo())
msg := fmt.Sprintf("sort path: %s, %s, please clean up the disk and retry", sortPath, d.UsageInfo())
return dbterror.ErrIngestCheckEnvFailed.FastGenByArgs(msg)
}
return nil
}
Expand Down
1 change: 1 addition & 0 deletions errno/errcode.go
Original file line number Diff line number Diff line change
Expand Up @@ -1127,6 +1127,7 @@ const (
ErrColumnInChange = 8245
ErrDDLSetting = 8246
ErrIngestFailed = 8247
ErrIngestCheckEnvFailed = 8256

ErrCannotPauseDDLJob = 8260
ErrCannotResumeDDLJob = 8261
Expand Down
1 change: 1 addition & 0 deletions errno/errname.go
Original file line number Diff line number Diff line change
Expand Up @@ -1119,6 +1119,7 @@ var MySQLErrName = map[uint16]*mysql.ErrMessage{
ErrPartitionColumnStatsMissing: mysql.Message("Build global-level stats failed due to missing partition-level column stats: %s, please run analyze table to refresh columns of all partitions", nil),
ErrDDLSetting: mysql.Message("Error happened when %s DDL: %s", nil),
ErrIngestFailed: mysql.Message("Ingest failed: %s", nil),
ErrIngestCheckEnvFailed: mysql.Message("Check ingest environment failed: %s", nil),
ErrNotSupportedWithSem: mysql.Message("Feature '%s' is not supported when security enhanced mode is enabled", nil),

ErrPlacementPolicyCheck: mysql.Message("Placement policy didn't meet the constraint, reason: %s", nil),
Expand Down
5 changes: 5 additions & 0 deletions errors.toml
Original file line number Diff line number Diff line change
Expand Up @@ -1466,6 +1466,11 @@ error = '''
Ingest failed: %s
'''

["ddl:8256"]
error = '''
Check ingest environment failed: %s
'''

["ddl:8260"]
error = '''
Job [%v] can't be paused: %s
Expand Down
21 changes: 18 additions & 3 deletions tests/realtikvtest/addindextest/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,9 +433,9 @@ func TestAddIndexBackfillLostUpdate(t *testing.T) {
d := dom.DDL()
originalCallback := d.GetHook()
defer d.SetHook(originalCallback)
callback := &callback.TestDDLCallback{}
hook := &callback.TestDDLCallback{}
var runDML bool
callback.OnJobRunAfterExported = func(job *model.Job) {
hook.OnJobRunAfterExported = func(job *model.Job) {
if t.Failed() || runDML {
return
}
Expand Down Expand Up @@ -470,10 +470,25 @@ func TestAddIndexBackfillLostUpdate(t *testing.T) {
_, err = tk1.Exec("commit;")
assert.NoError(t, err)
}
d.SetHook(callback)
d.SetHook(hook)
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/mockDMLExecutionStateBeforeImport", "1*return"))
tk.MustExec("alter table t add unique index idx(b);")
tk.MustExec("admin check table t;")
tk.MustQuery("select * from t;").Check(testkit.Rows("1 2 1"))
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/mockDMLExecutionStateBeforeImport"))
}

func TestAddIndexPreCheckFailed(t *testing.T) {
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("drop database if exists addindexlit;")
tk.MustExec("create database addindexlit;")
tk.MustExec("use addindexlit;")
tk.MustExec(`set global tidb_ddl_enable_fast_reorg=on;`)

tk.MustExec("create table t(id int primary key, b int, k int);")
tk.MustExec("insert into t values (1, 1, 1);")
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/ddl/ingest/mockIngestCheckEnvFailed", "return"))
tk.MustGetErrMsg("alter table t add index idx(b);", "[ddl:8256]Check ingest environment failed: mock error")
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/ddl/ingest/mockIngestCheckEnvFailed"))
}
2 changes: 2 additions & 0 deletions util/dbterror/ddl_terror.go
Original file line number Diff line number Diff line change
Expand Up @@ -403,6 +403,8 @@ var (
ErrDDLSetting = ClassDDL.NewStd(mysql.ErrDDLSetting)
// ErrIngestFailed returns when the DDL ingest job is failed.
ErrIngestFailed = ClassDDL.NewStd(mysql.ErrIngestFailed)
// ErrIngestCheckEnvFailed returns when the DDL ingest env is failed to init.
ErrIngestCheckEnvFailed = ClassDDL.NewStd(mysql.ErrIngestCheckEnvFailed)

// ErrColumnInChange indicates there is modification on the column in parallel.
ErrColumnInChange = ClassDDL.NewStd(mysql.ErrColumnInChange)
Expand Down

0 comments on commit 7f016ce

Please sign in to comment.