Skip to content

Commit

Permalink
ddl: handle create writer error for index ingest operator (#53916)
Browse files Browse the repository at this point in the history
close #53791
  • Loading branch information
tangenta authored Jun 11, 2024
1 parent 5c0d737 commit 0b54071
Show file tree
Hide file tree
Showing 4 changed files with 19 additions and 0 deletions.
1 change: 1 addition & 0 deletions pkg/ddl/backfilling_operators.go
Original file line number Diff line number Diff line change
Expand Up @@ -595,6 +595,7 @@ func NewIndexIngestOperator(
writer, err := engines[i].CreateWriter(writerID)
if err != nil {
logutil.Logger(ctx).Error("create index ingest worker failed", zap.Error(err))
ctx.onError(err)
return nil
}
writers = append(writers, writer)
Expand Down
6 changes: 6 additions & 0 deletions pkg/ddl/ingest/env.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import (
"path/filepath"
"strconv"

"github.com/pingcap/failpoint"
"github.com/pingcap/tidb/pkg/config"
"github.com/pingcap/tidb/pkg/ddl/logutil"
"github.com/pingcap/tidb/pkg/lightning/log"
Expand Down Expand Up @@ -68,6 +69,11 @@ func InitGlobalLightningEnv(filterProcessingJobIDs FilterProcessingJobIDsFunc) {
} else {
memTotal = memTotal / 2
}
failpoint.Inject("setMemTotalInMB", func(val failpoint.Value) {
//nolint: forcetypeassert
i := val.(int)
memTotal = uint64(i) * size.MB
})
LitBackCtxMgr = NewLitBackendCtxMgr(sortPath, memTotal, filterProcessingJobIDs)
litRLimit = util.GenRLimit("ddl-ingest")
LitInitialized = true
Expand Down
1 change: 1 addition & 0 deletions tests/realtikvtest/addindextest3/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ go_test(
"//pkg/table",
"//pkg/table/tables",
"//pkg/testkit",
"//pkg/testkit/testfailpoint",
"//pkg/util/chunk",
"//tests/realtikvtest",
"@com_github_ngaut_pools//:pools",
Expand Down
11 changes: 11 additions & 0 deletions tests/realtikvtest/addindextest3/functional_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
"github.com/pingcap/tidb/pkg/ddl/ingest"
"github.com/pingcap/tidb/pkg/parser/model"
"github.com/pingcap/tidb/pkg/testkit"
"github.com/pingcap/tidb/pkg/testkit/testfailpoint"
"github.com/pingcap/tidb/tests/realtikvtest"
"github.com/stretchr/testify/require"
"github.com/tikv/client-go/v2/tikv"
Expand Down Expand Up @@ -102,3 +103,13 @@ func TestBackendCtxConcurrentUnregister(t *testing.T) {
wg.Wait()
ingest.LitBackCtxMgr.Unregister(1)
}

func TestMockMemoryUsedUp(t *testing.T) {
testfailpoint.Enable(t, "github.com/pingcap/tidb/pkg/ddl/ingest/setMemTotalInMB", "return(1100)")
store := realtikvtest.CreateMockStoreAndSetup(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test;")
tk.MustExec("create table t (c int, c2 int, c3 int, c4 int);")
tk.MustExec("insert into t values (1,1,1,1), (2,2,2,2), (3,3,3,3);")
tk.MustGetErrMsg("alter table t add index i(c), add index i2(c2);", "[ddl:8247]Ingest failed: memory used up")
}

0 comments on commit 0b54071

Please sign in to comment.