Skip to content

Commit

Permalink
remove aggregate replacement.
Browse files Browse the repository at this point in the history
  • Loading branch information
gatorsmile committed Mar 10, 2016
1 parent 6cf6f44 commit 44326f1
Show file tree
Hide file tree
Showing 2 changed files with 9 additions and 20 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -315,6 +315,10 @@ object SetOperationPushDown extends Rule[LogicalPlan] with PredicateHelper {
* - LeftSemiJoin
*/
object ColumnPruning extends Rule[LogicalPlan] {
private def sameOutput(output1: Seq[Attribute], output2: Seq[Attribute]): Boolean =
output1.size == output2.size &&
output1.zip(output2).forall(pair => pair._1.semanticEquals(pair._2))

def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// Prunes the unused columns from project list of Project/Aggregate/Expand
case p @ Project(_, p2: Project) if (p2.outputSet -- p.references).nonEmpty =>
Expand Down Expand Up @@ -375,11 +379,7 @@ object ColumnPruning extends Rule[LogicalPlan] {
case p @ Project(projectList, child) if sameOutput(child.output, p.output) => child

// Eliminate no-op Window
case w: Window if sameOutput(w.child.output, w.output) => w.child

// Convert Aggregate to Project if no aggregate function exists
case a: Aggregate if !containsAggregates(a.expressions) =>
Project(a.aggregateExpressions, a.child)
case w: Window if w.windowExpressions.isEmpty => w.child

// Can't prune the columns on LeafNode
case p @ Project(_, l: LeafNode) => p
Expand Down Expand Up @@ -411,18 +411,6 @@ object ColumnPruning extends Rule[LogicalPlan] {
} else {
c
}

private def sameOutput(output1: Seq[Attribute], output2: Seq[Attribute]): Boolean =
output1.size == output2.size &&
output1.zip(output2).forall(pair => pair._1.semanticEquals(pair._2))

private def isAggregateExpression(e: Expression): Boolean = {
e.isInstanceOf[AggregateExpression] || e.isInstanceOf[Grouping] || e.isInstanceOf[GroupingID]
}

private def containsAggregates(exprs: Seq[Expression]): Boolean = {
exprs.exists(_.find(isAggregateExpression).nonEmpty)
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -184,7 +184,8 @@ class ColumnPruningSuite extends PlanTest {
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer =
testRelation
.select('a).analyze
.select('a)
.groupBy('a)('a).analyze

comparePlans(optimized, correctAnswer)
}
Expand All @@ -201,7 +202,7 @@ class ColumnPruningSuite extends PlanTest {
val correctAnswer =
testRelation
.select('a)
.select('a as 'c).analyze
.groupBy('a)('a as 'c).analyze

comparePlans(optimized, correctAnswer)
}
Expand Down Expand Up @@ -270,7 +271,7 @@ class ColumnPruningSuite extends PlanTest {
UnspecifiedFrame)).as('window)).select('a, 'c)

val correctAnswer =
input.select('a, 'c).analyze
input.select('a, 'c, 'd).groupBy('a, 'c, 'd)('a, 'c).analyze

val optimized = Optimize.execute(originalQuery.analyze)

Expand Down

0 comments on commit 44326f1

Please sign in to comment.