Skip to content

Commit

Permalink
planner: remove the limitation that predicates can't be pushed throug…
Browse files Browse the repository at this point in the history
…h `Projection` on `TableDual` (#51329) (#52164)

close #50614
  • Loading branch information
time-and-fate authored Mar 28, 2024
1 parent ca6a857 commit d3e6a29
Show file tree
Hide file tree
Showing 4 changed files with 77 additions and 46 deletions.
2 changes: 1 addition & 1 deletion planner/core/issuetest/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -6,6 +6,6 @@ go_test(
srcs = ["planner_issue_test.go"],
flaky = True,
race = "on",
shard_count = 9,
shard_count = 10,
deps = ["//testkit"],
)
112 changes: 72 additions & 40 deletions planner/core/issuetest/planner_issue_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,53 +154,53 @@ func TestIssue47881(t *testing.T) {
tk.MustExec("create table t2(id int,name varchar(10),name1 varchar(10),name2 varchar(10),`date1` date);")
tk.MustExec("insert into t2 values(1,'tt','ttt','tttt','2099-12-31'),(2,'dd','ddd','dddd','2099-12-31');")
rs := tk.MustQuery(`WITH bzzs AS (
SELECT
count(1) AS bzn
FROM
SELECT
count(1) AS bzn
FROM
t c
),
),
tmp1 AS (
SELECT
t1.*
FROM
t1
LEFT JOIN bzzs ON 1 = 1
WHERE
name IN ('tt')
SELECT
t1.*
FROM
t1
LEFT JOIN bzzs ON 1 = 1
WHERE
name IN ('tt')
AND bzn <> 1
),
),
tmp2 AS (
SELECT
tmp1.*,
date('2099-12-31') AS endate
FROM
SELECT
tmp1.*,
date('2099-12-31') AS endate
FROM
tmp1
),
),
tmp3 AS (
SELECT
*
FROM
tmp2
WHERE
endate > CURRENT_DATE
UNION ALL
SELECT
'1' AS id,
'ss' AS name,
'sss' AS name1,
'ssss' AS name2,
date('2099-12-31') AS endate
FROM
bzzs t1
WHERE
SELECT
*
FROM
tmp2
WHERE
endate > CURRENT_DATE
UNION ALL
SELECT
'1' AS id,
'ss' AS name,
'sss' AS name1,
'ssss' AS name2,
date('2099-12-31') AS endate
FROM
bzzs t1
WHERE
bzn = 1
)
SELECT
c2.id,
c3.id
FROM
t2 db
LEFT JOIN tmp3 c2 ON c2.id = '1'
)
SELECT
c2.id,
c3.id
FROM
t2 db
LEFT JOIN tmp3 c2 ON c2.id = '1'
LEFT JOIN tmp3 c3 ON c3.id = '1';`)
rs.Check(testkit.Rows("1 1", "1 1"))
}
Expand All @@ -216,3 +216,35 @@ func TestIssue48969(t *testing.T) {
tk.MustExec("update test2 set value=0 where test2.id in (select * from v1);")
tk.MustQuery("select * from test2").Check(testkit.Rows("1 0", "2 0", "3 0", "4 4", "5 5"))
}

func TestIssue50614(t *testing.T) {
store := testkit.CreateMockStore(t)
tk := testkit.NewTestKit(t, store)
tk.MustExec("use test")
tk.MustExec("drop table if exists tt")
tk.MustExec("create table tt(a bigint, b bigint, c bigint, d bigint, e bigint, primary key(c,d));")
tk.MustQuery("explain format = brief " +
"update tt, (select 1 as c1 ,2 as c2 ,3 as c3, 4 as c4 union all select 2,3,4,5 union all select 3,4,5,6) tmp " +
"set tt.a=tmp.c1, tt.b=tmp.c2 " +
"where tt.c=tmp.c3 and tt.d=tmp.c4 and (tt.c,tt.d) in ((11,111),(22,222),(33,333),(44,444));").Check(
testkit.Rows(
"Update N/A root N/A",
"└─Projection 0.00 root test.tt.a, test.tt.b, test.tt.c, test.tt.d, test.tt.e, Column#18, Column#19, Column#20, Column#21",
" └─Projection 0.00 root test.tt.a, test.tt.b, test.tt.c, test.tt.d, test.tt.e, Column#18, Column#19, Column#20, Column#21",
" └─IndexJoin 0.00 root inner join, inner:TableReader, outer key:Column#20, Column#21, inner key:test.tt.c, test.tt.d, equal cond:eq(Column#20, test.tt.c), eq(Column#21, test.tt.d), other cond:or(or(and(eq(Column#20, 11), eq(test.tt.d, 111)), and(eq(Column#20, 22), eq(test.tt.d, 222))), or(and(eq(Column#20, 33), eq(test.tt.d, 333)), and(eq(Column#20, 44), eq(test.tt.d, 444)))), or(or(and(eq(test.tt.c, 11), eq(Column#21, 111)), and(eq(test.tt.c, 22), eq(Column#21, 222))), or(and(eq(test.tt.c, 33), eq(Column#21, 333)), and(eq(test.tt.c, 44), eq(Column#21, 444))))",
" ├─Union(Build) 0.00 root ",
" │ ├─Projection 0.00 root Column#6, Column#7, Column#8, Column#9",
" │ │ └─Projection 0.00 root 1->Column#6, 2->Column#7, 3->Column#8, 4->Column#9",
" │ │ └─TableDual 0.00 root rows:0",
" │ ├─Projection 0.00 root Column#10, Column#11, Column#12, Column#13",
" │ │ └─Projection 0.00 root 2->Column#10, 3->Column#11, 4->Column#12, 5->Column#13",
" │ │ └─TableDual 0.00 root rows:0",
" │ └─Projection 0.00 root Column#14, Column#15, Column#16, Column#17",
" │ └─Projection 0.00 root 3->Column#14, 4->Column#15, 5->Column#16, 6->Column#17",
" │ └─TableDual 0.00 root rows:0",
" └─TableReader(Probe) 0.00 root data:Selection",
" └─Selection 0.00 cop[tikv] or(or(and(eq(test.tt.c, 11), eq(test.tt.d, 111)), and(eq(test.tt.c, 22), eq(test.tt.d, 222))), or(and(eq(test.tt.c, 33), eq(test.tt.d, 333)), and(eq(test.tt.c, 44), eq(test.tt.d, 444)))), or(or(eq(test.tt.c, 11), eq(test.tt.c, 22)), or(eq(test.tt.c, 33), eq(test.tt.c, 44))), or(or(eq(test.tt.d, 111), eq(test.tt.d, 222)), or(eq(test.tt.d, 333), eq(test.tt.d, 444)))",
" └─TableRangeScan 0.00 cop[tikv] table:tt range: decided by [eq(test.tt.c, Column#20) eq(test.tt.d, Column#21)], keep order:false, stats:pseudo",
),
)
}
4 changes: 4 additions & 0 deletions planner/core/logical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -1142,9 +1142,13 @@ type LogicalMaxOneRow struct {
}

// LogicalTableDual represents a dual table plan.
// Note that sometimes we don't set schema for LogicalTableDual (most notably in buildTableDual()), which means
// outputting 0/1 row with zero column. This semantic may be different from your expectation sometimes but should not
// cause any actual problems now.
type LogicalTableDual struct {
logicalSchemaProducer

// RowCount could only be 0 or 1.
RowCount int
}

Expand Down
5 changes: 0 additions & 5 deletions planner/core/rule_predicate_push_down.go
Original file line number Diff line number Diff line change
Expand Up @@ -508,11 +508,6 @@ func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression
return predicates, child
}
}
if len(p.children) == 1 {
if _, isDual := p.children[0].(*LogicalTableDual); isDual {
return predicates, p
}
}
for _, cond := range predicates {
substituted, hasFailed, newFilter := expression.ColumnSubstituteImpl(cond, p.Schema(), p.Exprs, true)
if substituted && !hasFailed && !expression.HasGetSetVarFunc(newFilter) {
Expand Down

0 comments on commit d3e6a29

Please sign in to comment.