Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

executor: fix deadlock in dml statement with cte when oom panic action was triggered #49192

Merged
merged 6 commits into from
Dec 5, 2023
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
27 changes: 18 additions & 9 deletions pkg/executor/cte.go
Original file line number Diff line number Diff line change
Expand Up @@ -113,15 +113,24 @@ func (e *CTEExec) Next(ctx context.Context, req *chunk.Chunk) (err error) {

// Close implements the Executor interface.
func (e *CTEExec) Close() (err error) {
e.producer.resTbl.Lock()
if !e.producer.closed {
// closeProducer() only close seedExec and recursiveExec, will not touch resTbl.
// It means you can still read resTbl after call closeProducer().
// You can even call all three functions(openProducer/produce/closeProducer) in CTEExec.Next().
// Separating these three function calls is only to follow the abstraction of the volcano model.
err = e.producer.closeProducer()
}
e.producer.resTbl.Unlock()
func() {
e.producer.resTbl.Lock()
defer e.producer.resTbl.Unlock()
if !e.producer.closed {
failpoint.Inject("mock_cte_exec_panic_avoid_deadlock", func(v failpoint.Value) {
ok := v.(bool)
if ok {
// mock an oom panic, returning ErrMemoryExceedForQuery for error identification in recovery work.
panic(exeerrors.ErrMemoryExceedForQuery)
}
})
// closeProducer() only close seedExec and recursiveExec, will not touch resTbl.
// It means you can still read resTbl after call closeProducer().
// You can even call all three functions(openProducer/produce/closeProducer) in CTEExec.Next().
// Separating these three function calls is only to follow the abstraction of the volcano model.
err = e.producer.closeProducer()
}
}()
if err != nil {
return err
}
Expand Down
31 changes: 31 additions & 0 deletions pkg/executor/cte_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,37 @@ import (
"github.com/stretchr/testify/require"
)

func TestCTEIssue49096(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)

tk.MustExec("use test;")
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/executor/mock_cte_exec_panic_avoid_deadlock", "return(true)"))
defer func() {
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/executor/mock_cte_exec_panic_avoid_deadlock"))
}()
insertStr := "insert into t1 values(0)"
rowNum := 10
vals := make([]int, rowNum)
vals[0] = 0
for i := 1; i < rowNum; i++ {
v := rand.Intn(100)
vals[i] = v
insertStr += fmt.Sprintf(", (%d)", v)
}
tk.MustExec("drop table if exists t1, t2;")
tk.MustExec("create table t1(c1 int);")
tk.MustExec("create table t2(c1 int);")
tk.MustExec(insertStr)
// should be insert statement, otherwise it couldn't step int resetCTEStorageMap in handleNoDelay func.
sql := "insert into t2 with cte1 as ( " +
"select c1 from t1) " +
"select c1 from cte1 natural join (select * from cte1 where c1 > 0) cte2 order by c1;"
err := tk.ExecToErr(sql)
require.NotNil(t, err)
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())
}

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

Expand Down
1 change: 1 addition & 0 deletions pkg/util/cteutil/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -10,6 +10,7 @@ go_library(
"//pkg/util/chunk",
"//pkg/util/disk",
"//pkg/util/memory",
"//pkg/util/syncutil",
"@com_github_pingcap_errors//:errors",
],
)
Expand Down
5 changes: 2 additions & 3 deletions pkg/util/cteutil/storage.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,13 +15,12 @@
package cteutil

import (
"sync"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/disk"
"github.com/pingcap/tidb/pkg/util/memory"
"github.com/pingcap/tidb/pkg/util/syncutil"
)

var _ Storage = &StorageRC{}
Expand Down Expand Up @@ -99,7 +98,7 @@ type StorageRC struct {
refCnt int
chkSize int
iter int
mu sync.Mutex
mu syncutil.Mutex
done bool
}

Expand Down