Skip to content

Commit

Permalink
Merge #42718
Browse files Browse the repository at this point in the history
42718: opt: push Limit through Left Joins r=RaduBerinde a=RaduBerinde

If we limit the results of a left join, we can also limit the join's
left input (because each left row generates at least one output row).
This is useful with a common pattern of queries where we have a "core"
query followed by a series of left joins that populate the rows
with extra information.

Release note (performance improvement): we generate better plans in
many cases where the query has LEFT / RIGHT JOINs and also has LIMIT.

Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
  • Loading branch information
craig[bot] and RaduBerinde committed Nov 26, 2019
2 parents d7a73c7 + 7973158 commit f97dc13
Show file tree
Hide file tree
Showing 3 changed files with 737 additions and 9 deletions.
60 changes: 60 additions & 0 deletions pkg/sql/opt/norm/rules/limit.opt
Original file line number Diff line number Diff line change
Expand Up @@ -115,3 +115,63 @@ $input
)
$private
)

# PushLimitIntoLeftJoin pushes a Limit into the left input of a left join. Since
# the left join creates an output row for each left input row, we only need that
# many rows from that input. We can only do this if the limit ordering refers
# only to the left input columns. We also check that the cardinality of the left
# input is more than the limit, to prevent repeated applications of the rule.
[PushLimitIntoLeftJoin, Normalize]
(Limit
$input:(LeftJoin
$left:*
$right:*
$on:*
$private:*
)
(Const $limit:*) & ^(LimitGeMaxRows $limit $left)
$ordering:* & (HasColsInOrdering $left $ordering)
)
=>
(Limit
(LeftJoin
(Limit
$left
(Const $limit)
(PruneOrdering $ordering (OutputCols $left))
)
$right
$on
$private
)
(Const $limit)
$ordering
)

# PushLimitIntoRightJoin is symmetric with PushLimitIntoLeftJoin.
[PushLimitIntoRightJoin, Normalize]
(Limit
$input:(RightJoin
$left:*
$right:*
$on:*
$private:*
)
(Const $limit:*) & ^(LimitGeMaxRows $limit $right)
$ordering:* & (HasColsInOrdering $right $ordering)
)
=>
(Limit
(RightJoin
$left
(Limit
$right
(Const $limit)
(PruneOrdering $ordering (OutputCols $right))
)
$on
$private
)
(Const $limit)
$ordering
)
20 changes: 12 additions & 8 deletions pkg/sql/opt/norm/testdata/rules/decorrelate
Original file line number Diff line number Diff line change
Expand Up @@ -4108,23 +4108,27 @@ values
├── cardinality: [0 - 1]
├── key: ()
├── fd: ()-->(2,7,9,12)
├── left-join (hash)
├── right-join (hash)
│ ├── columns: i:2(int) y:7(int) true:9(bool) rownum:12(int!null)
│ ├── fd: (12)-->(2), ()~~>(9)
│ ├── fd: ()-->(2,12), ()~~>(9)
│ ├── limit hint: 1.00
│ ├── ordinality
│ │ ├── columns: i:2(int) rownum:12(int!null)
│ │ ├── key: (12)
│ │ ├── fd: (12)-->(2)
│ │ └── scan a
│ │ └── columns: i:2(int)
│ ├── project
│ │ ├── columns: true:9(bool!null) y:7(int)
│ │ ├── fd: ()-->(9)
│ │ ├── scan xy
│ │ │ └── columns: y:7(int)
│ │ └── projections
│ │ └── true [type=bool]
│ ├── ordinality
│ │ ├── columns: i:2(int) rownum:12(int!null)
│ │ ├── cardinality: [0 - 1]
│ │ ├── key: ()
│ │ ├── fd: ()-->(2,12)
│ │ └── scan a
│ │ ├── columns: i:2(int)
│ │ ├── limit: 1
│ │ ├── key: ()
│ │ └── fd: ()-->(2)
│ └── filters
│ └── y = i [type=bool, outer=(2,7), constraints=(/2: (/NULL - ]; /7: (/NULL - ]), fd=(2)==(7), (7)==(2)]
└── const: 1 [type=int]
Expand Down
Loading

0 comments on commit f97dc13

Please sign in to comment.