Skip to content

Commit

Permalink
opt: add rule to fold two grouping operators into one
Browse files Browse the repository at this point in the history
Previously, the optimizer could not fold two grouping operators into
a single equivalent grouping operator.

This patch adds a rule that effects this transformation under the
following conditions:
1. All of the outer aggregates are aggregating on the output columns
   of the inner aggregates.
2. Every inner-outer aggregation pair can be replaced with an
   equivalent single aggregate.
3. The inner grouping columns functionally determine the outer
   grouping columns.
4. Both grouping operators are unordered.

As an example, the following query pairs are equivalent:
```
SELECT sum(t) FROM (SELECT sum(b) FROM ab GROUP BY a) AS g(t);
SELECT sum(b) FROM ab;

SELECT max(t) FROM (SELECT max(b) FROM ab GROUP BY a) AS g(t);
SELECT max(b) FROM ab;

SELECT sum_int(t) FROM (SELECT count(b) FROM ab GROUP BY a) AS g(t);
SELECT count(b) FROM ab;
```
This situation is rare in direct SQL queries, but can arise when
composing views and queries.

Release note (sql change): The optimizer can now fold two grouping
operators together when they are aggregating over functions like sum.
  • Loading branch information
DrewKimball committed Jun 1, 2020
1 parent 8f7cd61 commit 3806f86
Show file tree
Hide file tree
Showing 9 changed files with 459 additions and 15 deletions.
10 changes: 5 additions & 5 deletions pkg/sql/opt/norm/general_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -320,11 +320,11 @@ func (c *CustomFuncs) IsBoundBy(src opt.Expr, cols opt.ColSet) bool {
return c.OuterCols(src).SubsetOf(cols)
}

// IsDeterminedBy returns true if all outer references in the source expression
// are bound by the closure of the given columns according to the functional
// dependencies of the input expression.
func (c *CustomFuncs) IsDeterminedBy(src opt.Expr, cols opt.ColSet, input memo.RelExpr) bool {
return input.Relational().FuncDeps.InClosureOf(c.OuterCols(src), cols)
// ColsAreDeterminedBy returns true if the given columns are functionally
// determined by the "in" ColSet according to the functional dependencies of the
// input expression.
func (c *CustomFuncs) ColsAreDeterminedBy(cols, in opt.ColSet, input memo.RelExpr) bool {
return input.Relational().FuncDeps.InClosureOf(cols, in)
}

// AreProjectionsCorrelated returns true if any element in the projections
Expand Down
71 changes: 71 additions & 0 deletions pkg/sql/opt/norm/groupby_funcs.go
Original file line number Diff line number Diff line change
Expand Up @@ -308,3 +308,74 @@ func (c *CustomFuncs) areRowsDistinct(

return true
}

// CanMergeAggs returns true if the given inner and outer AggregationsExprs can
// be replaced with a single equivalent AggregationsExpr.
func (c *CustomFuncs) CanMergeAggs(innerAggs, outerAggs memo.AggregationsExpr) bool {
// Create a mapping from the output ColumnID of each inner aggregate to its
// operator type.
innerColsToAggOps := map[opt.ColumnID]opt.Operator{}
for i := range innerAggs {
innerAgg := innerAggs[i].Agg
if !opt.IsAggregateOp(innerAgg) {
// Aggregate can't be an AggFilter or AggDistinct.
return false
}
innerColsToAggOps[innerAggs[i].Col] = innerAgg.Op()
}

for i := range outerAggs {
outerAgg := outerAggs[i].Agg
if !opt.IsAggregateOp(outerAgg) {
// Aggregate can't be an AggFilter or AggDistinct.
return false
}
if outerAgg.ChildCount() != 1 {
// There are no valid inner-outer aggregate pairs for which the ChildCount
// of the outer is not equal to one.
return false
}
input, ok := outerAgg.Child(0).(*memo.VariableExpr)
if !ok {
// The outer aggregate does not directly aggregate on a column.
return false
}
innerOp, ok := innerColsToAggOps[input.Col]
if !ok {
// This outer aggregate does not reference an inner aggregate.
return false
}
if !opt.AggregatesCanMerge(innerOp, outerAgg.Op()) {
// There is no single aggregate that can replace this pair.
return false
}
}
return true
}

// MergeAggs returns an AggregationsExpr that is equivalent to the two given
// AggregationsExprs. MergeAggs will panic if CanMergeAggs is false.
func (c *CustomFuncs) MergeAggs(innerAggs, outerAggs memo.AggregationsExpr) memo.AggregationsExpr {
// Create a mapping from the output ColumnIDs of the inner aggregates to their
// indices in innerAggs.
innerColsToAggs := map[opt.ColumnID]int{}
for i := range innerAggs {
innerColsToAggs[innerAggs[i].Col] = i
}

newAggs := make(memo.AggregationsExpr, len(outerAggs))
for i := range outerAggs {
// For each outer aggregate, construct a new aggregate that takes the Agg
// field of the referenced inner aggregate and the Col field of the outer
// aggregate. This works because CanMergeAggs has already verified that
// every inner-outer aggregate pair forms a valid decomposition for the
// inner aggregate. In most cases, the inner and outer aggregates are the
// same, but in the count and count-rows cases the inner aggregate must
// be used (see opt.aggregatesCanMerge for details). The column from the
// outer aggregate has to be used to preserve logical equivalency.
inputCol := outerAggs[i].Agg.Child(0).(*memo.VariableExpr).Col
innerAgg := innerAggs[innerColsToAggs[inputCol]].Agg
newAggs[i] = c.f.ConstructAggregationsItem(innerAgg, outerAggs[i].Col)
}
return newAggs
}
88 changes: 88 additions & 0 deletions pkg/sql/opt/norm/rules/groupby.opt
Original file line number Diff line number Diff line change
Expand Up @@ -304,3 +304,91 @@
(ReplaceAggregationsItem $aggregations $item (CountRows))
$groupingPrivate
)

# FoldGroupingOperators folds two grouping operators into one equivalent
# operator. As an example, the following pairs of queries are equivalent:
#
# SELECT sum(t) FROM (SELECT sum(b) FROM ab GROUP BY a) AS g(t);
# SELECT sum(b) FROM ab;
#
# SELECT max(t) FROM (SELECT max(b) FROM ab GROUP BY a) AS g(t);
# SELECT max(b) FROM ab;
#
# SELECT sum_int(t) FROM (SELECT count(b) FROM ab GROUP BY a) AS g(t);
# SELECT count(b) FROM ab;
#
# This transformation is possible when the following conditions are met:
#
# 1. All of the outer aggregates are aggregating on the output columns of the
# inner aggregates.
# 2. All of the inner-outer aggregate pairs can be replaced with an equivalent
# single aggregate. (See the AggregatesCanMerge comment in operator.go).
# 3. The grouping columns of the inner operator functionally determine the
# grouping columns of the outer operator according to the functional
# dependencies of the input of the inner operator.
# 4. Both grouping operators are unordered.
#
# Why is it sufficient for the inner grouping columns to functionally determine
# the outer grouping columns?
# * Duplicate values in the determinant ("from" side) imply duplicate values in
# the dependent ("to" side).
# * Grouping on the determinant will not remove unique values from the
# determinant. Therefore, the grouping will not remove unique values from the
# dependent, by the properties of functional dependencies.
# * Grouping on the dependent will simply reduce the dependent to its unique
# values.
# * Therefore, grouping on the dependent produces the same final groups as
# grouping on the dependent after grouping on the determinant.
# * Conditions #2 and #4 guarantee that the aggregates produce the same result
# regardless of how the grouping is accomplished, as long as the same groups
# result in the end.
#
# Take the following table as an example:
#
# r a b
# -----
# 1 4 3
# 2 4 3
# 3 2 3
# 4 2 3
# 5 6 5
# 6 6 5
#
# Its functional dependencies: key(r), r-->(a, b), a-->(b)
#
# Here are some examples of possible groupings taking the sum over the "r"
# column:
#
# Grouping by a: SUM(1, 2), SUM(3, 4), SUM(5, 6)
# Grouping by b: SUM(1, 2, 3, 4), SUM(5, 6)
# Grouping by a then b: SUM(SUM(1, 2), SUM(3, 4)), SUM(SUM(5, 6))
#
# Rows can always be grouped together by subsequent groupings, but they can
# never be "ungrouped". Grouping on a does not group any rows together that
# would not also be grouped by b.
#
# This situation is rare in direct SQL queries, but can arise when composing
# views and queries.
[FoldGroupingOperators, Normalize]
(GroupBy | ScalarGroupBy
(GroupBy
$innerInput:*
$innerAggs:*
$innerGrouping:* & (IsUnorderedGrouping $innerGrouping)
)
$outerAggs:*
$outerGrouping:* &
(IsUnorderedGrouping $outerGrouping) &
(ColsAreDeterminedBy
$outerGroupingCols:(GroupingCols $outerGrouping)
(GroupingCols $innerGrouping)
$innerInput
) &
(CanMergeAggs $innerAggs $outerAggs)
)
=>
((OpName)
$innerInput
(MergeAggs $innerAggs $outerAggs)
(MakeGrouping $outerGroupingCols (EmptyOrdering))
)
4 changes: 2 additions & 2 deletions pkg/sql/opt/norm/rules/window.opt
Original file line number Diff line number Diff line change
Expand Up @@ -58,8 +58,8 @@ $input
$filters:[
...
$item:* &
(IsDeterminedBy
$item
(ColsAreDeterminedBy
(OuterCols $item)
$partitionCols:(WindowPartition $private)
$input
)
Expand Down
Loading

0 comments on commit 3806f86

Please sign in to comment.