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: Check if the query is killed when joining rows in HashJoin #20270

Merged
merged 13 commits into from
Oct 28, 2020
22 changes: 22 additions & 0 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -561,6 +561,17 @@ func (e *HashJoinExec) getNewJoinResult(workerID uint) (bool, *hashjoinWorkerRes

func (e *HashJoinExec) join2Chunk(workerID uint, probeSideChk *chunk.Chunk, hCtx *hashContext, joinResult *hashjoinWorkerResult,
selected []bool) (ok bool, _ *hashjoinWorkerResult) {
killed := atomic.LoadUint32(&e.ctx.GetSessionVars().Killed) == 1
failpoint.Inject("killedInJoin2Chunk", func(val failpoint.Value) {
if val.(bool) {
killed = true
}
})
if killed {
joinResult.err = ErrQueryInterrupted
return false, joinResult
}
qw4990 marked this conversation as resolved.
Show resolved Hide resolved

var err error
selected, err = expression.VectorizedFilter(e.ctx, e.outerFilter, chunk.NewIterator4Chunk(probeSideChk), selected)
if err != nil {
Expand Down Expand Up @@ -601,6 +612,17 @@ func (e *HashJoinExec) join2Chunk(workerID uint, probeSideChk *chunk.Chunk, hCtx

// join2ChunkForOuterHashJoin joins chunks when using the outer to build a hash table (refer to outer hash join)
func (e *HashJoinExec) join2ChunkForOuterHashJoin(workerID uint, probeSideChk *chunk.Chunk, hCtx *hashContext, joinResult *hashjoinWorkerResult) (ok bool, _ *hashjoinWorkerResult) {
killed := atomic.LoadUint32(&e.ctx.GetSessionVars().Killed) == 1
failpoint.Inject("killedInJoin2ChunkForOuterHashJoin", func(val failpoint.Value) {
if val.(bool) {
killed = true
}
})
if killed {
joinResult.err = ErrQueryInterrupted
return false, joinResult
}

hCtx.initHash(probeSideChk.NumRows())
for _, i := range hCtx.keyColIdx {
err := codec.HashChunkColumns(e.rowContainer.sc, hCtx.hashVals, probeSideChk, hCtx.allTypes[i], i, hCtx.buf, hCtx.hasNull)
Expand Down
23 changes: 23 additions & 0 deletions executor/join_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,9 @@ import (

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"

qw4990 marked this conversation as resolved.
Show resolved Hide resolved
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/executor"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/util"
Expand Down Expand Up @@ -2321,3 +2323,24 @@ func (s *testSuiteJoinSerial) TestExplainAnalyzeJoin(c *C) {
c.Assert(rows[0][0], Matches, "HashJoin.*")
c.Assert(rows[0][5], Matches, "time:.*, loops:.*, build_hash_table:{total:.*, fetch:.*, build:.*}, probe:{concurrency:5, total:.*, max:.*, probe:.*, fetch:.*}")
}

func (s *testSuiteJoinSerial) TestIssue20270(c *C) {
tk := testkit.NewTestKitWithInit(c, s.store)
failpoint.Enable("github.com/pingcap/tidb/executor/killedInJoin2Chunk", "return(true)")
tk.MustExec("drop table if exists t;")
tk.MustExec("drop table if exists t1;")
tk.MustExec("create table t(c1 int, c2 int)")
tk.MustExec("create table t1(c1 int, c2 int)")
tk.MustExec("insert into t values(1,1),(2,2)")
tk.MustExec("insert into t1 values(2,3),(4,4)")
err := tk.QueryToErr("select /*+ TIDB_HJ(t, t1) */ * from t left join t1 on t.c1 = t1.c1 where t.c1 = 1 or t1.c2 > 20")
c.Assert(err, Equals, executor.ErrQueryInterrupted)
failpoint.Disable("github.com/pingcap/tidb/executor/killedInJoin2Chunk")
plannercore.ForceUseOuterBuild4Test = true
failpoint.Enable("github.com/pingcap/tidb/executor/killedInJoin2ChunkForOuterHashJoin", "return(true)")
tk.MustExec("insert into t1 values(1,30),(2,40)")
err = tk.QueryToErr("select /*+ TIDB_HJ(t, t1) */ * from t left outer join t1 on t.c1 = t1.c1 where t.c1 = 1 or t1.c2 > 20")
c.Assert(err, Equals, executor.ErrQueryInterrupted)
failpoint.Disable("github.com/pingcap/tidb/executor/killedInJoin2ChunkForOuterHashJoin")
plannercore.ForceUseOuterBuild4Test = false
qw4990 marked this conversation as resolved.
Show resolved Hide resolved
}