Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
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 @@ -59,7 +59,7 @@ object DefaultOptimizer extends Optimizer {
ConstantFolding,
LikeSimplification,
BooleanSimplification,
RemoveDispensable,
RemoveDispensableExpressions,
SimplifyFilters,
SimplifyCasts,
SimplifyCaseConversionExpressions) ::
Expand Down Expand Up @@ -660,49 +660,49 @@ object PushPredicateThroughGenerate extends Rule[LogicalPlan] with PredicateHelp
case filter @ Filter(condition, g: Generate) =>
// Predicates that reference attributes produced by the `Generate` operator cannot
// be pushed below the operator.
val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition {
conjunct => conjunct.references subsetOf g.child.outputSet
val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond =>
cond.references subsetOf g.child.outputSet
Copy link
Contributor Author

Choose a reason for hiding this comment

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

conjunct doesn't make sense - "and" is the conjunction, and the individual conditions are not.

}
if (pushDown.nonEmpty) {
val pushDownPredicate = pushDown.reduce(And)
val withPushdown = Generate(g.generator, join = g.join, outer = g.outer,
val newGenerate = Generate(g.generator, join = g.join, outer = g.outer,
g.qualifier, g.generatorOutput, Filter(pushDownPredicate, g.child))
stayUp.reduceOption(And).map(Filter(_, withPushdown)).getOrElse(withPushdown)
if (stayUp.isEmpty) newGenerate else Filter(stayUp.reduce(And), newGenerate)
} else {
filter
}
}
}

/**
* Push [[Filter]] operators through [[Aggregate]] operators. Parts of the predicate that reference
* attributes which are subset of group by attribute set of [[Aggregate]] will be pushed beneath,
* and the rest should remain above.
* Push [[Filter]] operators through [[Aggregate]] operators, iff the filters reference only
* non-aggregate attributes (typically literals or grouping expressions).
*/
object PushPredicateThroughAggregate extends Rule[LogicalPlan] with PredicateHelper {

def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case filter @ Filter(condition,
aggregate @ Aggregate(groupingExpressions, aggregateExpressions, grandChild)) =>

def hasAggregate(expression: Expression): Boolean = expression match {
case agg: AggregateExpression => true
case other => expression.children.exists(hasAggregate)
}
// Create a map of Alias for expressions that does not have AggregateExpression
val aliasMap = AttributeMap(aggregateExpressions.collect {
case a: Alias if !hasAggregate(a.child) => (a.toAttribute, a.child)
case filter @ Filter(condition, aggregate: Aggregate) =>
// Find all the aliased expressions in the aggregate list that don't include any actual
// AggregateExpression, and create a map from the alias to the expression
val aliasMap = AttributeMap(aggregate.aggregateExpressions.collect {
case a: Alias if a.child.find(_.isInstanceOf[AggregateExpression]).isEmpty =>
(a.toAttribute, a.child)
})

val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { conjunct =>
val replaced = replaceAlias(conjunct, aliasMap)
replaced.references.subsetOf(grandChild.outputSet) && replaced.deterministic
// For each filter, expand the alias and check if the filter can be evaluated using
// attributes produced by the aggregate operator's child operator.
val (pushDown, stayUp) = splitConjunctivePredicates(condition).partition { cond =>
val replaced = replaceAlias(cond, aliasMap)
replaced.references.subsetOf(aggregate.child.outputSet) && replaced.deterministic
}

if (pushDown.nonEmpty) {
val pushDownPredicate = pushDown.reduce(And)
val replaced = replaceAlias(pushDownPredicate, aliasMap)
val withPushdown = aggregate.copy(child = Filter(replaced, grandChild))
stayUp.reduceOption(And).map(Filter(_, withPushdown)).getOrElse(withPushdown)
val newAggregate = aggregate.copy(child = Filter(replaced, aggregate.child))
// If there is no more filter to stay up, just eliminate the filter.
// Otherwise, create "Filter(stayUp) <- Aggregate <- Filter(pushDownPredicate)".
if (stayUp.isEmpty) newAggregate else Filter(stayUp.reduce(And), newAggregate)
} else {
filter
}
Expand All @@ -714,7 +714,7 @@ object PushPredicateThroughAggregate extends Rule[LogicalPlan] with PredicateHel
* evaluated using only the attributes of the left or right side of a join. Other
* [[Filter]] conditions are moved into the `condition` of the [[Join]].
*
* And also Pushes down the join filter, where the `condition` can be evaluated using only the
* And also pushes down the join filter, where the `condition` can be evaluated using only the
* attributes of the left or right side of sub query when applicable.
*
* Check https://cwiki.apache.org/confluence/display/Hive/OuterJoinBehavior for more details
Expand Down Expand Up @@ -821,7 +821,7 @@ object SimplifyCasts extends Rule[LogicalPlan] {
/**
* Removes nodes that are not necessary.
*/
object RemoveDispensable extends Rule[LogicalPlan] {
object RemoveDispensableExpressions extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case UnaryPositive(child) => child
case PromotePrecision(child) => child
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -734,7 +734,7 @@ class FilterPushdownSuite extends PlanTest {
comparePlans(optimized, correctAnswer)
}

test("aggregate: don't push down filters which is nondeterministic") {
test("aggregate: don't push down filters that are nondeterministic") {
val originalQuery = testRelation
.select('a, 'b)
.groupBy('a)('a + Rand(10) as 'aa, count('b) as 'c, Rand(11).as("rnd"))
Expand Down