Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/295#discussion_r11279472
  
    --- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala ---
    @@ -55,61 +54,135 @@ case class Aggregate(
     
       override def otherCopyArgs = sc :: Nil
     
    +  // HACK: Generators don't correctly preserve their output through 
serializations so we grab
    +  // out child's output attributes statically here.
    +  val childOutput = child.output
    +
       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, 
childOutput).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.length)
    +    var i = 0
    +    while (i < computedAggregates.length) {
    +      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.length) {
    +            buffer(i).update(currentRow)
    +            i += 1
    +          }
    +        }
    +        val resultProjection = new Projection(resultExpressions, 
computedSchema)
    +        val aggregateResults = new 
GenericMutableRow(computedAggregates.length)
    +
    +        var i = 0
    +        while (i < buffer.length) {
    +          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, childOutput)
    +
    +        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.length) {
    +            currentBuffer(i).update(currentRow)
    +            i += 1
    +          }
    +        }
    +
    +        new Iterator[Row] {
    +          private[this] val hashTableIter = hashTable.entrySet().iterator()
    +          private[this] val aggregateResults = new 
GenericMutableRow(computedAggregates.length)
    +          private[this] val resultProjection =
    +            new MutableProjection(resultExpressions, computedSchema ++ 
namedGroups.map(_._2))
    +          private[this] val joinedRow = new JoinedRow
    +
    +          override final def hasNext: Boolean = hashTableIter.hasNext
    +
    +          override final def next(): Row = {
    +            val currentEntry = hashTableIter.next()
    +            val currentGroup = currentEntry.getKey
    +            val currentBuffer = currentEntry.getValue
    +
    +            var i = 0
    +            while (i < currentBuffer.length) {
    +              aggregateResults(i) = currentBuffer(i).apply(EmptyRow)
    --- End diff --
    
    add one line comment explaining this jsut returns the agg result (no need 
for a row)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to