Skip to content

Commit

Permalink
sql: fix the subquery limit optimization
Browse files Browse the repository at this point in the history
A while ago the HP was equipped with an optimization: when a subquery
is planned for EXISTS or "max 1 row" (scalar context), a LIMIT is
applied on its data source. This ensures that the data source does not
fetch more rows than strictly necessary to determine the subquery
result:

- for EXISTS, only 0 or 1 row are needed to decide the boolean;
- for scalar contexts, only 0, 1 or 2 rows are needed to decide the
  outcome 0 or 2 yield an error, only 1 gets a valid result.

This optimization was temporarily broken for the scalar case when
`max1row` was introduced (when local exec was subsumed by distsql),
because the limit was remaining "on top" of `max1row` and not
propagated down. This patch places it "under" so it gets propagated
again.

Release note (performance improvement): subqueries used with EXISTS or
as a scalar value now avoid fetching more rows than needed to decide
the outcome.
  • Loading branch information
knz committed Feb 12, 2019
1 parent 235c81c commit 14cce3d
Show file tree
Hide file tree
Showing 18 changed files with 214 additions and 116 deletions.
6 changes: 3 additions & 3 deletions pkg/sql/logictest/testdata/planner_test/needed_columns
Original file line number Diff line number Diff line change
Expand Up @@ -129,9 +129,9 @@ root · ·
│ id @S1 · ·
│ original sql (SELECT 2 AS x FROM (SELECT 3 AS s)) · ·
│ exec mode one row · ·
└── limit · · (x) x=CONST
count 2 · ·
└── max1row · · (x) x=CONST
└── max1row · · (x) x=CONST
└── limit · · (x) x=CONST
count 2 · ·
└── render · · (x) x=CONST
│ render 0 2 · ·
└── render · · (s[omitted]) ·
Expand Down
31 changes: 16 additions & 15 deletions pkg/sql/logictest/testdata/planner_test/subquery
Original file line number Diff line number Diff line change
Expand Up @@ -12,9 +12,9 @@ root · ·
│ id @S1
│ original sql (SELECT 1)
│ exec mode one row
└── limit · ·
count 2
└── max1row · ·
└── max1row · ·
└── limit · ·
count 2
└── render · ·
└── emptyrow · ·

Expand All @@ -36,9 +36,9 @@ root · ·
│ id @S1
│ original sql (SELECT 42)
│ exec mode one row
└── limit · ·
count 2
└── max1row · ·
└── max1row · ·
└── limit · ·
count 2
└── render · ·
└── emptyrow · ·

Expand Down Expand Up @@ -73,13 +73,14 @@ root · ·
│ id @S1
│ original sql (SELECT a FROM abc)
│ exec mode one row
└── limit · ·
count 2
└── max1row · ·
└── max1row · ·
└── limit · ·
count 2
└── render · ·
└── scan · ·
· table abc@primary
· spans ALL
· limit 2

query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM abc WHERE a = (SELECT max(a) FROM abc WHERE EXISTS(SELECT * FROM abc WHERE c=a+3))
Expand All @@ -103,9 +104,9 @@ root · ·
│ id @S2 · ·
│ original sql (SELECT max(a) FROM abc WHERE EXISTS (SELECT * FROM abc WHERE c = (a + 3))) · ·
│ exec mode one row · ·
└── limit · · (max) ·
count 2 · ·
└── max1row · · (max) ·
└── max1row · · (max) ·
└── limit · · (max) ·
count 2 · ·
└── group · · (max) ·
│ aggregate 0 max(a) · ·
│ scalar · · ·
Expand Down Expand Up @@ -156,9 +157,9 @@ root · ·
│ id @S1
│ original sql (SELECT 2)
│ exec mode one row
└── limit · ·
count 2
└── max1row · ·
└── max1row · ·
└── limit · ·
count 2
└── render · ·
└── emptyrow · ·

Expand Down
42 changes: 22 additions & 20 deletions pkg/sql/opt/exec/execbuilder/testdata/srfs
Original file line number Diff line number Diff line change
Expand Up @@ -167,26 +167,28 @@ render · ·
query TTTTT
EXPLAIN (VERBOSE) SELECT generate_series((SELECT unnest(ARRAY[x, y]) FROM xy), z) FROM xz
----
root · · (generate_series) ·
├── render · · (generate_series) ·
│ │ render 0 generate_series · ·
│ └── project set · · (z, generate_series) ·
│ │ render 0 generate_series(@S1, @1) · ·
│ └── scan · · (z) ·
│ table xz@primary · ·
│ spans ALL · ·
└── subquery · · (generate_series) ·
│ id @S1 · ·
│ original sql (SELECT unnest(ARRAY[x, y]) FROM xy) · ·
│ exec mode one row · ·
└── max1row · · (unnest) ·
└── render · · (unnest) ·
│ render 0 unnest · ·
└── project set · · (x, y, unnest) ·
│ render 0 unnest(ARRAY[@1, @2]) · ·
└── scan · · (x, y) ·
· table xy@primary · ·
· spans ALL · ·
root · · (generate_series) ·
├── render · · (generate_series) ·
│ │ render 0 generate_series · ·
│ └── project set · · (z, generate_series) ·
│ │ render 0 generate_series(@S1, @1) · ·
│ └── scan · · (z) ·
│ table xz@primary · ·
│ spans ALL · ·
└── subquery · · (generate_series) ·
│ id @S1 · ·
│ original sql (SELECT unnest(ARRAY[x, y]) FROM xy) · ·
│ exec mode one row · ·
└── max1row · · (unnest) ·
└── render · · (unnest) ·
│ render 0 unnest · ·
└── limit · · (x, y, unnest) ·
│ count 2 · ·
└── project set · · (x, y, unnest) ·
│ render 0 unnest(ARRAY[@1, @2]) · ·
└── scan · · (x, y) ·
· table xy@primary · ·
· spans ALL · ·

# Regression test for #24676.
statement ok
Expand Down
11 changes: 7 additions & 4 deletions pkg/sql/opt/exec/execbuilder/testdata/subquery
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ root · ·
│ id @S1
│ original sql (SELECT 42)
│ exec mode one row
└── limit · ·
count 2
└── max1row · ·
└── max1row · ·
└── limit · ·
count 2
└── render · ·
└── emptyrow · ·

Expand All @@ -43,6 +43,7 @@ root · ·
└── scan · ·
· table abc@primary
· spans ALL
· limit 1

query TTTTT
EXPLAIN (VERBOSE) SELECT * FROM abc WHERE a = (SELECT max(a) FROM abc WHERE EXISTS(SELECT * FROM abc WHERE c=a+3))
Expand All @@ -56,7 +57,9 @@ root · ·
│ │ id @S1 · ·
│ │ original sql EXISTS (SELECT * FROM abc WHERE c = (a + 3)) · ·
│ │ exec mode exists · ·
│ └── scan · · (a, b, c) ·
│ └── limit · · (a, b, c) ·
│ │ count 1 · ·
│ └── scan · · (a, b, c) ·
│ table abc@primary · ·
│ spans ALL · ·
│ filter c = (a + 3) · ·
Expand Down
20 changes: 20 additions & 0 deletions pkg/sql/opt/norm/custom_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -370,6 +370,12 @@ func (c *CustomFuncs) PruneOrdering(
return ordCopy
}

// EmptyOrdering returns a pseudo-choice that does not require any
// ordering.
func (c *CustomFuncs) EmptyOrdering() physical.OrderingChoice {
return physical.OrderingChoice{}
}

// -----------------------------------------------------------------------
//
// Filter functions
Expand Down Expand Up @@ -1237,6 +1243,20 @@ func (c *CustomFuncs) MakeOrderedGrouping(
return &memo.GroupingPrivate{GroupingCols: groupingCols, Ordering: ordering}
}

// IsSubqueryLimited indicates whether a limit was pushed under the subquery
// already. See the rules Max1RowLimitScan and ExistsLimitScan.
func (c *CustomFuncs) IsSubqueryLimited(sub *memo.SubqueryPrivate) bool {
return sub.SubqueryLimited
}

// SetSubqueryLimited specifies that the subquery has a limit set already.
// This prevents the rules Max1RowLimitScan and ExistsLimitScan from
// applying twice.
func (c *CustomFuncs) SetSubqueryLimited(sub *memo.SubqueryPrivate) *memo.SubqueryPrivate {
sub.SubqueryLimited = true
return sub
}

// ----------------------------------------------------------------------
//
// Numeric Rules
Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/opt/norm/rules/limit.opt
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,25 @@
# limit.opt contains normalization rules for the Limit and Offset operators.
# =============================================================================

# Max1RowLimit inserts a LIMIT 2 "under" Max1Row so as to save resources
# to make the Max1Row determination.
#
# This rule is defined here instead of max1row.opt so that it runs
# before the limit elimination rules below.
#
# We avoid this rule if the query is decorrelated because the
# decorrelation rules get confused by the presence of a limit.
# (It will be worth re-considering this when a general-purpose apply
# operator is supported - in that case it can be definitely worthwhile
# pushing down a LIMIT 2 to limit the amount of work done on every row.)
[Max1RowLimitScan, Normalize]
(Max1Row
$input:* & ^(HasZeroOrOneRow $input) & ^(HasOuterCols $input)
$subqueryPrivate:* & ^(IsSubqueryLimited $subqueryPrivate))
=>
((OpName)
(Limit $input (Const 2) (EmptyOrdering))
(SetSubqueryLimited $subqueryPrivate))

# EliminateLimit discards a Limit operator if its constant limit is greater than
# or equal to the maximum number of rows that can be returned by the input. In
Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/opt/norm/rules/scalar.opt
Original file line number Diff line number Diff line change
Expand Up @@ -167,6 +167,24 @@
[EliminateExistsGroupBy, Normalize]
(Exists (GroupBy | DistinctOn $input:*) $subqueryPrivate:*) => (Exists $input $subqueryPrivate)

# ExistsLimit inserts a LIMIT 1 "under" Exists so as to save resources
# to make the EXISTS determination.
#
# We avoid this rule if the query is decorrelated because the
# decorrelation rules get confused by the presence of a limit.
# (It will be worth re-considering this when a general-purpose apply
# operator is supported - in that case it can be definitely worthwhile
# pushing down a LIMIT 1 to limit the amount of work done on every row.)
[ExistsLimit, Normalize]
(Exists
$input:^(Project | GroupBy | DistinctOn) & ^(HasOuterCols $input) & ^(HasZeroOrOneRow $input)
$subqueryPrivate:* & ^(IsSubqueryLimited $subqueryPrivate))
=>
((OpName)
(Limit $input (Const 1) (EmptyOrdering))
(SetSubqueryLimited $subqueryPrivate))


# NormalizeJSONFieldAccess transforms field access into a containment with a
# simpler LHS. This allows inverted index constraints to be generated in some
# cases.
Expand Down
Loading

0 comments on commit 14cce3d

Please sign in to comment.