Skip to content

Commit 0c7659c

Browse files
authored
executor: fix deadlock in dml statement with cte when oom panic action was triggered (#49192)
close #49096
1 parent e053c27 commit 0c7659c

File tree

4 files changed

+52
-12
lines changed

4 files changed

+52
-12
lines changed

pkg/executor/cte.go

+18-9
Original file line numberDiff line numberDiff line change
@@ -113,15 +113,24 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {
113113

114114
// Close implements the Executor interface.
115115
func (e *CTEExec) Close() (err error) {
116-
e.producer.resTbl.Lock()
117-
if !e.producer.closed {
118-
// closeProducer() only close seedExec and recursiveExec, will not touch resTbl.
119-
// It means you can still read resTbl after call closeProducer().
120-
// You can even call all three functions(openProducer/produce/closeProducer) in CTEExec.Next().
121-
// Separating these three function calls is only to follow the abstraction of the volcano model.
122-
err = e.producer.closeProducer()
123-
}
124-
e.producer.resTbl.Unlock()
116+
func() {
117+
e.producer.resTbl.Lock()
118+
defer e.producer.resTbl.Unlock()
119+
if !e.producer.closed {
120+
failpoint.Inject("mock_cte_exec_panic_avoid_deadlock", func(v failpoint.Value) {
121+
ok := v.(bool)
122+
if ok {
123+
// mock an oom panic, returning ErrMemoryExceedForQuery for error identification in recovery work.
124+
panic(exeerrors.ErrMemoryExceedForQuery)
125+
}
126+
})
127+
// closeProducer() only close seedExec and recursiveExec, will not touch resTbl.
128+
// It means you can still read resTbl after call closeProducer().
129+
// You can even call all three functions(openProducer/produce/closeProducer) in CTEExec.Next().
130+
// Separating these three function calls is only to follow the abstraction of the volcano model.
131+
err = e.producer.closeProducer()
132+
}
133+
}()
125134
if err != nil {
126135
return err
127136
}

pkg/executor/cte_test.go

+31
Original file line numberDiff line numberDiff line change
@@ -27,6 +27,37 @@ import (
2727
"github.com/stretchr/testify/require"
2828
)
2929

30+
func TestCTEIssue49096(t *testing.T) {
31+
store := testkit.CreateMockStore(t)
32+
tk := testkit.NewTestKit(t, store)
33+
34+
tk.MustExec("use test;")
35+
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mock_cte_exec_panic_avoid_deadlock", "return(true)"))
36+
defer func() {
37+
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mock_cte_exec_panic_avoid_deadlock"))
38+
}()
39+
insertStr := "insert into t1 values(0)"
40+
rowNum := 10
41+
vals := make([]int, rowNum)
42+
vals[0] = 0
43+
for i := 1; i < rowNum; i++ {
44+
v := rand.Intn(100)
45+
vals[i] = v
46+
insertStr += fmt.Sprintf(", (%d)", v)
47+
}
48+
tk.MustExec("drop table if exists t1, t2;")
49+
tk.MustExec("create table t1(c1 int);")
50+
tk.MustExec("create table t2(c1 int);")
51+
tk.MustExec(insertStr)
52+
// should be insert statement, otherwise it couldn't step int resetCTEStorageMap in handleNoDelay func.
53+
sql := "insert into t2 with cte1 as ( " +
54+
"select c1 from t1) " +
55+
"select c1 from cte1 natural join (select * from cte1 where c1 > 0) cte2 order by c1;"
56+
err := tk.ExecToErr(sql)
57+
require.NotNil(t, err)
58+
require.Equal(t, "[executor:8175]Your query has been cancelled due to exceeding the allowed memory limit for a single SQL query. Please try narrowing your query scope or increase the tidb_mem_quota_query limit and try again.[conn=%d]", err.Error())
59+
}
60+
3061
func TestSpillToDisk(t *testing.T) {
3162
store := testkit.CreateMockStore(t)
3263

pkg/util/cteutil/BUILD.bazel

+1
Original file line numberDiff line numberDiff line change
@@ -10,6 +10,7 @@ go_library(
1010
"//pkg/util/chunk",
1111
"//pkg/util/disk",
1212
"//pkg/util/memory",
13+
"//pkg/util/syncutil",
1314
"@com_github_pingcap_errors//:errors",
1415
],
1516
)

pkg/util/cteutil/storage.go

+2-3
Original file line numberDiff line numberDiff line change
@@ -15,13 +15,12 @@
1515
package cteutil
1616

1717
import (
18-
"sync"
19-
2018
"github.com/pingcap/errors"
2119
"github.com/pingcap/tidb/pkg/types"
2220
"github.com/pingcap/tidb/pkg/util/chunk"
2321
"github.com/pingcap/tidb/pkg/util/disk"
2422
"github.com/pingcap/tidb/pkg/util/memory"
23+
"github.com/pingcap/tidb/pkg/util/syncutil"
2524
)
2625

2726
var _ Storage = &StorageRC{}
@@ -99,7 +98,7 @@ type StorageRC struct {
9998
refCnt int
10099
chkSize int
101100
iter int
102-
mu sync.Mutex
101+
mu syncutil.Mutex
103102
done bool
104103
}
105104

0 commit comments

Comments
 (0)