Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
30 commits
Select commit Hold shift + click to select a range
84ba723
RemoveRedundantAggregates
tanelk Oct 12, 2020
14f3033
Merge branch 'master' into SPARK-33122
tanelk Oct 12, 2020
29701dc
Clearer naming
tanelk Oct 14, 2020
4ce0644
Clearer naming
tanelk Oct 14, 2020
ef64abf
Handle aliases
Oct 16, 2020
ca974c7
Merge branch 'master' into SPARK-33122
tanelk Oct 16, 2020
4bf08bb
UTs for non-deterministic cases
tanelk Oct 16, 2020
832ff02
Extract alias helper
tanelk Oct 21, 2020
6cdc43a
cleanup
Oct 21, 2020
2d63bb4
Extract AliasHelper
tanelk Oct 22, 2020
a82699e
Loosen the determinicity check
tanelk Oct 22, 2020
38d7007
Typo
tanelk Oct 22, 2020
fab0427
Keep old behaviour with attribute naming
tanelk Oct 22, 2020
67861f9
Merge branch 'master' into SPARK-33122
tanelk Oct 27, 2020
12d1bf4
Fix merge conflict
tanelk Oct 27, 2020
e396ce3
Fix merge conflict
tanelk Dec 18, 2020
6d68718
Fix merge conflict
tanelk Dec 18, 2020
0d86060
Fix failing test
tanelk Dec 19, 2020
33d6072
Remove non-deterministic restriction
tanelk Dec 19, 2020
3e3168a
Address comments
tanelk Dec 20, 2020
57af005
Add UTs for python funcs
tanelk Dec 21, 2020
f50048f
Merge branch 'SPARK-33122' of https://github.com/tanelk/spark into SP…
Jan 5, 2021
9219bca
Merge branch 'master' into SPARK-33122
Jan 5, 2021
c21dd52
Merge branch 'master' into SPARK-33122
Jan 5, 2021
b415194
upperRefsOnlyDeterministicNonAgg
tanelk Jan 11, 2021
797d48f
upperRefsOnlyDeterministicNonAgg
tanelk Jan 11, 2021
2b32f4a
Merge branch 'master' into SPARK-33122
tanelk Jan 16, 2021
e202987
Cleanup
tanelk Jan 16, 2021
37dc4b1
rerun plan stability
tanelk Jan 26, 2021
07e758d
Trigger build
tanelk Mar 19, 2021
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -3010,56 +3010,6 @@ class Analyzer(override val catalogManager: CatalogManager)
}
}

/**
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Just moved it outside of Analyzer, so it would be accessible from outside

* Pulls out nondeterministic expressions from LogicalPlan which is not Project or Filter,
* put them into an inner Project and finally project them away at the outer Project.
*/
object PullOutNondeterministic extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp {
case p if !p.resolved => p // Skip unresolved nodes.
case p: Project => p
case f: Filter => f

case a: Aggregate if a.groupingExpressions.exists(!_.deterministic) =>
val nondeterToAttr = getNondeterToAttr(a.groupingExpressions)
val newChild = Project(a.child.output ++ nondeterToAttr.values, a.child)
a.transformExpressions { case e =>
nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e)
}.copy(child = newChild)

// Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail)
// and we want to retain them inside the aggregate functions.
case m: CollectMetrics => m

// todo: It's hard to write a general rule to pull out nondeterministic expressions
// from LogicalPlan, currently we only do it for UnaryNode which has same output
// schema with its child.
case p: UnaryNode if p.output == p.child.output && p.expressions.exists(!_.deterministic) =>
val nondeterToAttr = getNondeterToAttr(p.expressions)
val newPlan = p.transformExpressions { case e =>
nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e)
}
val newChild = Project(p.child.output ++ nondeterToAttr.values, p.child)
Project(p.output, newPlan.withNewChildren(newChild :: Nil))
}

private def getNondeterToAttr(exprs: Seq[Expression]): Map[Expression, NamedExpression] = {
exprs.filterNot(_.deterministic).flatMap { expr =>
val leafNondeterministic = expr.collect {
case n: Nondeterministic => n
case udf: UserDefinedExpression if !udf.deterministic => udf
}
leafNondeterministic.distinct.map { e =>
val ne = e match {
case n: NamedExpression => n
case _ => Alias(e, "_nondeterministic")()
}
e -> ne
}
}.toMap
}
}

/**
* Set the seed for random number generation.
*/
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,74 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule

/**
* Pulls out nondeterministic expressions from LogicalPlan which is not Project or Filter,
* put them into an inner Project and finally project them away at the outer Project.
*/
object PullOutNondeterministic extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp applyLocally

val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = {
case p if !p.resolved => p // Skip unresolved nodes.
case p: Project => p
case f: Filter => f

case a: Aggregate if a.groupingExpressions.exists(!_.deterministic) =>
val nondeterToAttr = getNondeterToAttr(a.groupingExpressions)
val newChild = Project(a.child.output ++ nondeterToAttr.values, a.child)
a.transformExpressions { case e =>
nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e)
}.copy(child = newChild)

// Don't touch collect metrics. Top-level metrics are not supported (check analysis will fail)
// and we want to retain them inside the aggregate functions.
case m: CollectMetrics => m

// todo: It's hard to write a general rule to pull out nondeterministic expressions
// from LogicalPlan, currently we only do it for UnaryNode which has same output
// schema with its child.
case p: UnaryNode if p.output == p.child.output && p.expressions.exists(!_.deterministic) =>
val nondeterToAttr = getNondeterToAttr(p.expressions)
val newPlan = p.transformExpressions { case e =>
nondeterToAttr.get(e).map(_.toAttribute).getOrElse(e)
}
val newChild = Project(p.child.output ++ nondeterToAttr.values, p.child)
Project(p.output, newPlan.withNewChildren(newChild :: Nil))
}

private def getNondeterToAttr(exprs: Seq[Expression]): Map[Expression, NamedExpression] = {
exprs.filterNot(_.deterministic).flatMap { expr =>
val leafNondeterministic = expr.collect {
case n: Nondeterministic => n
case udf: UserDefinedExpression if !udf.deterministic => udf
}
leafNondeterministic.distinct.map { e =>
val ne = e match {
case n: NamedExpression => n
case _ => Alias(e, "_nondeterministic")()
}
e -> ne
}
}.toMap
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -111,6 +111,7 @@ abstract class Optimizer(catalogManager: CatalogManager)
RewriteCorrelatedScalarSubquery,
EliminateSerialization,
RemoveRedundantAliases,
RemoveRedundantAggregates,
UnwrapCastInBinaryComparison,
RemoveNoopOperators,
OptimizeUpdateFields,
Expand Down Expand Up @@ -488,6 +489,50 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = removeRedundantAliases(plan, AttributeSet.empty)
}

/**
* Remove redundant aggregates from a query plan. A redundant aggregate is an aggregate whose
* only goal is to keep distinct values, while its parent aggregate would ignore duplicate values.
*/
object RemoveRedundantAggregates extends Rule[LogicalPlan] with AliasHelper {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could you move this optimizer into a new file please, @tanelk ?

def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case upper @ Aggregate(_, _, lower: Aggregate) if lowerIsRedundant(upper, lower) =>
val aliasMap = getAliasMap(lower)

val newAggregate = upper.copy(
child = lower.child,
groupingExpressions = upper.groupingExpressions.map(replaceAlias(_, aliasMap)),
aggregateExpressions = upper.aggregateExpressions.map(
replaceAliasButKeepName(_, aliasMap))
)

// We might have introduces non-deterministic grouping expression
Copy link
Member

@dongjoon-hyun dongjoon-hyun Mar 20, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  • introduces -> introduced
  • expression -> expressions

if (newAggregate.groupingExpressions.exists(!_.deterministic)) {
PullOutNondeterministic.applyLocally.applyOrElse(newAggregate, identity[LogicalPlan])
} else {
newAggregate
}
}

private def lowerIsRedundant(upper: Aggregate, lower: Aggregate): Boolean = {
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit. Usually, isXXX is better and consistent with Apache Spark convention.

val upperHasNoAggregateExpressions = !upper.aggregateExpressions.exists(isAggregate)

lazy val upperRefsOnlyDeterministicNonAgg = upper.references.subsetOf(AttributeSet(
lower
.aggregateExpressions
.filter(_.deterministic)
.filter(!isAggregate(_))
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

- .filter(!isAggregate(_))
+ .filterNot(isAggregate)

.map(_.toAttribute)
))

upperHasNoAggregateExpressions && upperRefsOnlyDeterministicNonAgg
}

private def isAggregate(expr: Expression): Boolean = {
expr.find(e => e.isInstanceOf[AggregateExpression] ||
PythonUDF.isGroupedAggPandasUDF(e)).isDefined
}
}

/**
* Remove no-op operators from the query plan that do not make any modifications.
*/
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -621,7 +621,7 @@ case class Range(
*
* @param groupingExpressions expressions for grouping keys
* @param aggregateExpressions expressions for a project list, which could contain
* [[AggregateFunction]]s.
* [[AggregateExpression]]s.
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This caused some confusion while making this PR

*
* Note: Currently, aggregateExpressions is the project list of this Group by operator. Before
* separating projection from grouping and aggregate, we should avoid expression-level optimization
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,163 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.catalyst.optimizer

import org.apache.spark.api.python.PythonEvalType
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions.{Expression, PythonUDF}
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.types.IntegerType

class RemoveRedundantAggregatesSuite extends PlanTest {

object Optimize extends RuleExecutor[LogicalPlan] {
val batches = Batch("RemoveRedundantAggregates", FixedPoint(10),
RemoveRedundantAggregates) :: Nil
}

private def aggregates(e: Expression): Seq[Expression] = {
Seq(
count(e),
PythonUDF("pyUDF", null, IntegerType, Seq(e),
PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF, udfDeterministic = true)
)
}

test("Remove redundant aggregate") {
val relation = LocalRelation('a.int, 'b.int)
for (agg <- aggregates('b)) {
val query = relation
.groupBy('a)('a, agg)
.groupBy('a)('a)
.analyze
val expected = relation
.groupBy('a)('a)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, expected)
}
}

test("Remove 2 redundant aggregates") {
val relation = LocalRelation('a.int, 'b.int)
for (agg <- aggregates('b)) {
val query = relation
.groupBy('a)('a, agg)
.groupBy('a)('a)
.groupBy('a)('a)
.analyze
val expected = relation
.groupBy('a)('a)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, expected)
}
}

test("Remove redundant aggregate with different grouping") {
val relation = LocalRelation('a.int, 'b.int)
val query = relation
.groupBy('a, 'b)('a)
.groupBy('a)('a)
.analyze
val expected = relation
.groupBy('a)('a)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, expected)
}

test("Remove redundant aggregate with aliases") {
val relation = LocalRelation('a.int, 'b.int)
for (agg <- aggregates('b)) {
val query = relation
.groupBy('a + 'b)(('a + 'b) as 'c, agg)
.groupBy('c)('c)
.analyze
val expected = relation
.groupBy('a + 'b)(('a + 'b) as 'c)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, expected)
}
}

test("Remove redundant aggregate with non-deterministic upper") {
val relation = LocalRelation('a.int, 'b.int)
val query = relation
.groupBy('a)('a)
.groupBy('a)('a, rand(0) as 'c)
.analyze
val expected = relation
.groupBy('a)('a, rand(0) as 'c)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, expected)
}

test("Remove redundant aggregate with non-deterministic lower") {
val relation = LocalRelation('a.int, 'b.int)
val query = relation
.groupBy('a, 'c)('a, rand(0) as 'c)
.groupBy('a, 'c)('a, 'c)
.analyze
val expected = relation
.groupBy('a, 'c)('a, rand(0) as 'c)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm, shouldn't this expected be .groupBy('a)('a, rand(0) as 'c)? If this test passes currently then this PR would introduce a correctness issue.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thank you, good catch. I fixed this test, added another to check this case and added an extra condition to the optimizer.

Copy link
Contributor

@peter-toth peter-toth Jan 16, 2021

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

LGTM now, conflicts need to be resolved though.

.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, expected)
}

test("Keep non-redundant aggregate - upper has agg expression") {
val relation = LocalRelation('a.int, 'b.int)
for (agg <- aggregates('b)) {
val query = relation
.groupBy('a, 'b)('a, 'b)
// The count would change if we remove the first aggregate
.groupBy('a)('a, agg)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, query)
}
}

test("Keep non-redundant aggregate - upper references agg expression") {
val relation = LocalRelation('a.int, 'b.int)
for (agg <- aggregates('b)) {
val query = relation
.groupBy('a)('a, agg as 'c)
.groupBy('c)('c)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, query)
}
}

test("Keep non-redundant aggregate - upper references non-deterministic non-grouping") {
val relation = LocalRelation('a.int, 'b.int)
val query = relation
.groupBy('a)('a, ('a + rand(0)) as 'c)
.groupBy('a, 'c)('a, 'c)
.analyze
val optimized = Optimize.execute(query)
comparePlans(optimized, query)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -192,7 +192,7 @@ abstract class RemoveRedundantProjectsSuiteBase
|)
|""".stripMargin

Seq(("UNION", 2, 2), ("UNION ALL", 1, 2)).foreach { case (setOperation, enabled, disabled) =>
Seq(("UNION", 1, 2), ("UNION ALL", 1, 2)).foreach { case (setOperation, enabled, disabled) =>
val query = queryTemplate.format(setOperation)
assertProjectExec(query, enabled = enabled, disabled = disabled)
}
Expand Down