Skip to content

Commit

Permalink
planner: fix correctness of the correlated predicate push down for cte (
Browse files Browse the repository at this point in the history
#43759)

close #43645
  • Loading branch information
winoros committed May 13, 2023
1 parent cde395d commit 6caadca
Show file tree
Hide file tree
Showing 3 changed files with 31 additions and 3 deletions.
1 change: 1 addition & 0 deletions planner/core/issuetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -9,6 +9,7 @@ go_test(
],
flaky = True,
race = "on",
shard_count = 3,
deps = [
"//parser",
"//planner",
Expand Down
14 changes: 14 additions & 0 deletions planner/core/issuetest/planner_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,3 +69,17 @@ func TestIssue43461(t *testing.T) {

require.NotEqual(t, is.Columns, ts.Columns)
}

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

tk.MustExec("use test")
tk.MustExec("CREATE TABLE t1(id int,col1 varchar(10),col2 varchar(10),col3 varchar(10));")
tk.MustExec("CREATE TABLE t2(id int,col1 varchar(10),col2 varchar(10),col3 varchar(10));")
tk.MustExec("INSERT INTO t1 values(1,NULL,NULL,null),(2,NULL,NULL,null),(3,NULL,NULL,null);")
tk.MustExec("INSERT INTO t2 values(1,'a','aa','aaa'),(2,'b','bb','bbb'),(3,'c','cc','ccc');")

rs := tk.MustQuery("WITH tmp AS (SELECT t2.* FROM t2) select (SELECT tmp.col1 FROM tmp WHERE tmp.id=t1.id ) col1, (SELECT tmp.col2 FROM tmp WHERE tmp.id=t1.id ) col2, (SELECT tmp.col3 FROM tmp WHERE tmp.id=t1.id ) col3 from t1;")
rs.Sort().Check(testkit.Rows("a aa aaa", "b bb bbb", "c cc ccc"))
}
19 changes: 16 additions & 3 deletions planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -988,13 +988,26 @@ func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *lo
if !p.isOuterMostCTE {
return predicates, p.self
}
if len(predicates) == 0 {
pushedPredicates := make([]expression.Expression, len(predicates))
copy(pushedPredicates, predicates)
// The filter might change the correlated status of the cte.
// We forbid the push down that makes the change for now.
// Will support it later.
if !p.cte.IsInApply {
for i := len(pushedPredicates) - 1; i >= 0; i-- {
if len(expression.ExtractCorColumns(pushedPredicates[i])) == 0 {
continue
}
pushedPredicates = append(pushedPredicates[0:i], pushedPredicates[i+1:]...)
}
}
if len(pushedPredicates) == 0 {
p.cte.pushDownPredicates = append(p.cte.pushDownPredicates, expression.NewOne())
return predicates, p.self
}
newPred := make([]expression.Expression, 0, len(predicates))
for i := range predicates {
newPred = append(newPred, predicates[i].Clone())
for i := range pushedPredicates {
newPred = append(newPred, pushedPredicates[i].Clone())
ResolveExprAndReplace(newPred[i], p.cte.ColumnMap)
}
p.cte.pushDownPredicates = append(p.cte.pushDownPredicates, expression.ComposeCNFCondition(p.ctx, newPred...))
Expand Down

0 comments on commit 6caadca

Please sign in to comment.