Skip to content

Commit

Permalink
executor: fix hang when index join newInnerworker panic (#45890) (#46114
Browse files Browse the repository at this point in the history
)

close #45716
  • Loading branch information
ti-chi-bot authored Sep 7, 2023
1 parent 2d63b90 commit 5c47f11
Show file tree
Hide file tree
Showing 2 changed files with 26 additions and 2 deletions.
10 changes: 8 additions & 2 deletions executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,9 +188,10 @@ func (e *IndexLookUpJoin) startWorkers(ctx context.Context) {
innerCh := make(chan *lookUpJoinTask, concurrency)
e.workerWg.Add(1)
go e.newOuterWorker(resultCh, innerCh).run(workerCtx, e.workerWg)
e.workerWg.Add(concurrency)
for i := 0; i < concurrency; i++ {
go e.newInnerWorker(innerCh).run(workerCtx, e.workerWg)
innerWorker := e.newInnerWorker(innerCh)
e.workerWg.Add(1)
go innerWorker.run(workerCtx, e.workerWg)
}
}

Expand Down Expand Up @@ -232,6 +233,11 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork
lookup: e,
memTracker: memory.NewTracker(memory.LabelForIndexJoinInnerWorker, -1),
}
failpoint.Inject("inlNewInnerPanic", func(val failpoint.Value) {
if val.(bool) {
panic("test inlNewInnerPanic")
}
})
iw.memTracker.AttachTo(e.memTracker)
if len(copiedRanges) != 0 {
// We should not consume this memory usage in `iw.memTracker`. The
Expand Down
18 changes: 18 additions & 0 deletions executor/index_lookup_join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -493,3 +493,21 @@ func TestPartitionTableIndexJoinAndIndexReader(t *testing.T) {
tk.MustQuery("select /*+ TIDB_INLJ(t1, t2) */ t1.a from t t1, t t2 where t1.a=t2.b and " + cond).Sort().Check(result)
}
}

func TestIssue45716(t *testing.T) {
store := testkit.CreateMockStore(t)

tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("set tidb_mem_quota_query = 120000;")
tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1(a int, index(a));")
tk.MustExec("create table t2(a int, index(a));")
tk.MustExec("insert into t1 values (1), (2);")
tk.MustExec("insert into t2 values (1),(1),(2),(2);")

failpoint.Enable("github.com/pingcap/tidb/executor/inlNewInnerPanic", `return(true)`)
defer failpoint.Disable("github.com/pingcap/tidb/executor/inlNewInnerPanic")
err := tk.QueryToErr("select /*+ inl_join(t2) */ * from t1 join t2 on t1.a = t2.a;")
tk.MustContainErrMsg(err.Error(), "test inlNewInnerPanic")
}

0 comments on commit 5c47f11

Please sign in to comment.