Skip to content

Commit

Permalink
Rewrite Aggregate operator to stream input and use projections. Remov…
Browse files Browse the repository at this point in the history
…e unused local RDD functions implicits.
  • Loading branch information
marmbrus committed Apr 3, 2014
1 parent 5096f99 commit 7c13112
Show file tree
Hide file tree
Showing 2 changed files with 117 additions and 148 deletions.

This file was deleted.

165 changes: 117 additions & 48 deletions sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
Original file line number Diff line number Diff line change
Expand Up @@ -17,14 +17,13 @@

package org.apache.spark.sql.execution

import java.util.HashMap

import org.apache.spark.SparkContext
import org.apache.spark.sql.catalyst.errors._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.physical._

/* Implicit conversions */
import org.apache.spark.rdd.PartitionLocalRDDFunctions._

/**
* Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each
* group.
Expand All @@ -40,7 +39,7 @@ case class Aggregate(
groupingExpressions: Seq[Expression],
aggregateExpressions: Seq[NamedExpression],
child: SparkPlan)(@transient sc: SparkContext)
extends UnaryNode {
extends UnaryNode with NoBind {

override def requiredChildDistribution =
if (partial) {
Expand All @@ -57,59 +56,129 @@ case class Aggregate(

def output = aggregateExpressions.map(_.toAttribute)

/* Replace all aggregate expressions with spark functions that will compute the result. */
def createAggregateImplementations() = aggregateExpressions.map { agg =>
val impl = agg transform {
case a: AggregateExpression => a.newInstance
case class ComputedAggregate(
unbound: AggregateExpression, // Unbound aggregate used for result substitution
aggregate: AggregateExpression, // A bound copy of this aggregate used to create a buffer
resultAttribute: AttributeReference) // An attribute used to refer to the result of this agg

// A list of aggregates that need to be computed for each group.
@transient
lazy val computedAggregates = aggregateExpressions.flatMap { agg =>
agg.collect {
case a: AggregateExpression =>
ComputedAggregate(
a,
BindReferences.bindReference(a, child.output).asInstanceOf[AggregateExpression],
AttributeReference(s"aggResult:$a", a.dataType, nullable = true)())
}
}.toArray

// The schema of the result of all aggregate evaluations
@transient
lazy val computedSchema = computedAggregates.map(_.resultAttribute)

// Creates a new aggregate buffer for a group.
def newAggregateBuffer(): Array[AggregateFunction] = {
val buffer = new Array[AggregateFunction](computedAggregates.size)
var i = 0
while(i < computedAggregates.size) {
buffer(i) = computedAggregates(i).aggregate.newInstance
i += 1
}
buffer
}

val remainingAttributes = impl.collect { case a: Attribute => a }
// If any references exist that are not inside agg functions then the must be grouping exprs
// in this case we must rebind them to the grouping tuple.
if (remainingAttributes.nonEmpty) {
val unaliasedAggregateExpr = agg transform { case Alias(c, _) => c }

// An exact match with a grouping expression
val exactGroupingExpr = groupingExpressions.indexOf(unaliasedAggregateExpr) match {
case -1 => None
case ordinal => Some(BoundReference(ordinal, Alias(impl, "AGGEXPR")().toAttribute))
}
// Named attributes used to substitute grouping attributes into the final result.
@transient
lazy val namedGroups = groupingExpressions.map {
case ne: NamedExpression => ne -> ne.toAttribute
case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute
}

exactGroupingExpr.getOrElse(
sys.error(s"$agg is not in grouping expressions: $groupingExpressions"))
} else {
impl
// A map of substitutions that are used to insert the aggregate expressions and grouping
// expression into the final result expression.
@transient
lazy val resultMap =
(computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap

// Substituted version of aggregateExpressions expressions which are used to compute final
// output rows given a group and the result of all aggregate computations.
@transient
lazy val resultExpressions = aggregateExpressions.map { agg =>
agg.transform {
case e: Expression if resultMap.contains(e) => resultMap(e)
}
}

def execute() = attachTree(this, "execute") {
// TODO: If the child of it is an [[catalyst.execution.Exchange]],
// do not evaluate the groupingExpressions again since we have evaluated it
// in the [[catalyst.execution.Exchange]].
val grouped = child.execute().mapPartitions { iter =>
val buildGrouping = new Projection(groupingExpressions)
iter.map(row => (buildGrouping(row), row.copy()))
}.groupByKeyLocally()

val result = grouped.map { case (group, rows) =>
val aggImplementations = createAggregateImplementations()

// Pull out all the functions so we can feed each row into them.
val aggFunctions = aggImplementations.flatMap(_ collect { case f: AggregateFunction => f })

rows.foreach { row =>
aggFunctions.foreach(_.update(row))
if (groupingExpressions.isEmpty) {
child.execute().mapPartitions { iter =>
val buffer = newAggregateBuffer()
var currentRow: Row = null
while (iter.hasNext) {
currentRow = iter.next()
var i = 0
while (i < buffer.size) {
buffer(i).update(currentRow)
i += 1
}
}
val resultProjection = new Projection(resultExpressions, computedSchema)
val aggregateResults = new GenericMutableRow(computedAggregates.size)

var i = 0
while (i < buffer.size) {
aggregateResults(i) = buffer(i).apply(EmptyRow)
i += 1
}

Iterator(resultProjection(aggregateResults))
}
buildRow(aggImplementations.map(_.apply(group)))
}

// TODO: THIS BREAKS PIPELINING, DOUBLE COMPUTES THE ANSWER, AND USES TOO MUCH MEMORY...
if (groupingExpressions.isEmpty && result.count == 0) {
// When there there is no output to the Aggregate operator, we still output an empty row.
val aggImplementations = createAggregateImplementations()
sc.makeRDD(buildRow(aggImplementations.map(_.apply(null))) :: Nil)
} else {
result
child.execute().mapPartitions { iter =>
val hashTable = new HashMap[Row, Array[AggregateFunction]]
val groupingProjection = new MutableProjection(groupingExpressions, child.output)

var currentRow: Row = null
while (iter.hasNext) {
currentRow = iter.next()
val currentGroup = groupingProjection(currentRow)
var currentBuffer = hashTable.get(currentGroup)
if (currentBuffer == null) {
currentBuffer = newAggregateBuffer()
hashTable.put(currentGroup.copy(), currentBuffer)
}

var i = 0
while (i < currentBuffer.size) {
currentBuffer(i).update(currentRow)
i += 1
}
}

new Iterator[Row] {
private[this] val hashTableIter = hashTable.entrySet().iterator()
private[this] val aggregateResults = new GenericMutableRow(computedAggregates.size)
private[this] val resultProjection =
new MutableProjection(resultExpressions, computedSchema ++ namedGroups.map(_._2))
private[this] val joinedRow = new JoinedRow

final def hasNext: Boolean = hashTableIter.hasNext

final def next(): Row = {
val currentEntry = hashTableIter.next()
val currentGroup = currentEntry.getKey
val currentBuffer = currentEntry.getValue

var i = 0
while (i < currentBuffer.size) {
aggregateResults(i) = currentBuffer(i).apply(EmptyRow)
i += 1
}
resultProjection(joinedRow(aggregateResults, currentGroup))
}
}
}
}
}
}

0 comments on commit 7c13112

Please sign in to comment.