[SPARK-9240] [SQL] Hybrid aggregate operator using unsafe row

This PR adds a base aggregation iterator `AggregationIterator`, which is used 
to create `SortBasedAggregationIterator` (for sort-based aggregation) and 
`UnsafeHybridAggregationIterator` (first it tries hash-based aggregation and 
falls back to the sort-based aggregation (using external sorter) if we cannot 
allocate memory for the map). With these two iterators, we will not need 
existing iterators and I am removing those. Also, we can use a single physical 
`Aggregate` operator and it internally determines what iterators to used.

https://issues.apache.org/jira/browse/SPARK-9240

Author: Yin Huai <yh...@databricks.com>

Closes #7813 from yhuai/AggregateOperator and squashes the following commits:

e317e2b [Yin Huai] Remove unnecessary change.
74d93c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into 
AggregateOperator
ba6afbc [Yin Huai] Add a little bit more comments.
c9cf3b6 [Yin Huai] update
0f1b06f [Yin Huai] Remove unnecessary code.
21fd15f [Yin Huai] Remove unnecessary change.
964f88b [Yin Huai] Implement fallback strategy.
b1ea5cf [Yin Huai] wip
7fcbd87 [Yin Huai] Add a flag to control what iterator to use.
533d5b2 [Yin Huai] Prepare for fallback!
33b7022 [Yin Huai] wip
bd9282b [Yin Huai] UDAFs now supports UnsafeRow.
f52ee53 [Yin Huai] wip
3171f44 [Yin Huai] wip
d2c45a0 [Yin Huai] wip
f60cc83 [Yin Huai] Also check input schema.
af32210 [Yin Huai] Check iter.hasNext before we create an iterator because the 
constructor of the iterato will read at least one row from a non-empty input 
iter.
299008c [Yin Huai] First round cleanup.
3915bac [Yin Huai] Create a base iterator class for aggregation iterators and 
add the initial version of the hybrid iterator.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1ebd41b1
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1ebd41b1
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1ebd41b1

Branch: refs/heads/master
Commit: 1ebd41b141a95ec264bd2dd50f0fe24cd459035d
Parents: 98d6d9c
Author: Yin Huai <yh...@databricks.com>
Authored: Mon Aug 3 00:23:08 2015 -0700
Committer: Reynold Xin <r...@databricks.com>
Committed: Mon Aug 3 00:23:08 2015 -0700

----------------------------------------------------------------------
 .../expressions/aggregate/interfaces.scala      |  19 +-
 .../sql/execution/aggregate/Aggregate.scala     | 182 +++++
 .../aggregate/AggregationIterator.scala         | 490 ++++++++++++++
 .../SortBasedAggregationIterator.scala          | 236 +++++++
 .../UnsafeHybridAggregationIterator.scala       | 398 +++++++++++
 .../aggregate/aggregateOperators.scala          | 175 -----
 .../aggregate/sortBasedIterators.scala          | 664 -------------------
 .../spark/sql/execution/aggregate/udaf.scala    | 269 +++++++-
 .../spark/sql/execution/aggregate/utils.scala   |  99 +--
 .../spark/sql/execution/basicOperators.scala    |   1 -
 .../org/apache/spark/sql/SQLQuerySuite.scala    |  10 +-
 .../execution/SparkSqlSerializer2Suite.scala    |   9 +-
 .../hive/execution/AggregationQuerySuite.scala  | 118 ++--
 13 files changed, 1697 insertions(+), 973 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
index d08f553..4abfdfe 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala
@@ -110,7 +110,11 @@ abstract class AggregateFunction2
    * buffer value of `avg(x)` will be 0 and the position of the first buffer 
value of `avg(y)`
    * will be 2.
    */
-  var mutableBufferOffset: Int = 0
+  protected var mutableBufferOffset: Int = 0
+
+  def withNewMutableBufferOffset(newMutableBufferOffset: Int): Unit = {
+    mutableBufferOffset = newMutableBufferOffset
+  }
 
   /**
    * The offset of this function's start buffer value in the
@@ -126,7 +130,11 @@ abstract class AggregateFunction2
    * buffer value of `avg(x)` will be 1 and the position of the first buffer 
value of `avg(y)`
    * will be 3 (position 0 is used for the value of key`).
    */
-  var inputBufferOffset: Int = 0
+  protected var inputBufferOffset: Int = 0
+
+  def withNewInputBufferOffset(newInputBufferOffset: Int): Unit = {
+    inputBufferOffset = newInputBufferOffset
+  }
 
   /** The schema of the aggregation buffer. */
   def bufferSchema: StructType
@@ -195,11 +203,8 @@ abstract class AlgebraicAggregate extends 
AggregateFunction2 with Serializable w
   override def bufferSchema: StructType = 
StructType.fromAttributes(bufferAttributes)
 
   override def initialize(buffer: MutableRow): Unit = {
-    var i = 0
-    while (i < bufferAttributes.size) {
-      buffer(i + mutableBufferOffset) = initialValues(i).eval()
-      i += 1
-    }
+    throw new UnsupportedOperationException(
+      "AlgebraicAggregate's initialize should not be called directly")
   }
 
   override final def update(buffer: MutableRow, input: InternalRow): Unit = {

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala
new file mode 100644
index 0000000..cf568dc
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala
@@ -0,0 +1,182 @@
+/*
+ * 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.execution.aggregate
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.errors._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.physical.{UnspecifiedDistribution, 
ClusteredDistribution, AllTuples, Distribution}
+import org.apache.spark.sql.execution.{UnsafeFixedWidthAggregationMap, 
SparkPlan, UnaryNode}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * An Aggregate Operator used to evaluate [[AggregateFunction2]]. Based on the 
data types
+ * of the grouping expressions and aggregate functions, it determines if it 
uses
+ * sort-based aggregation and hybrid (hash-based with sort-based as the 
fallback) to
+ * process input rows.
+ */
+case class Aggregate(
+    requiredChildDistributionExpressions: Option[Seq[Expression]],
+    groupingExpressions: Seq[NamedExpression],
+    nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+    nonCompleteAggregateAttributes: Seq[Attribute],
+    completeAggregateExpressions: Seq[AggregateExpression2],
+    completeAggregateAttributes: Seq[Attribute],
+    initialInputBufferOffset: Int,
+    resultExpressions: Seq[NamedExpression],
+    child: SparkPlan)
+  extends UnaryNode {
+
+  private[this] val allAggregateExpressions =
+    nonCompleteAggregateExpressions ++ completeAggregateExpressions
+
+  private[this] val hasNonAlgebricAggregateFunctions =
+    
!allAggregateExpressions.forall(_.aggregateFunction.isInstanceOf[AlgebraicAggregate])
+
+  // Use the hybrid iterator if (1) unsafe is enabled, (2) the schemata of
+  // grouping key and aggregation buffer is supported; and (3) all
+  // aggregate functions are algebraic.
+  private[this] val supportsHybridIterator: Boolean = {
+    val aggregationBufferSchema: StructType =
+      StructType.fromAttributes(
+        allAggregateExpressions.flatMap(_.aggregateFunction.bufferAttributes))
+    val groupKeySchema: StructType =
+      StructType.fromAttributes(groupingExpressions.map(_.toAttribute))
+
+    val schemaSupportsUnsafe: Boolean =
+      
UnsafeFixedWidthAggregationMap.supportsAggregationBufferSchema(aggregationBufferSchema)
 &&
+        UnsafeProjection.canSupport(groupKeySchema)
+
+    // TODO: Use the hybrid iterator for non-algebric aggregate functions.
+    sqlContext.conf.unsafeEnabled && schemaSupportsUnsafe && 
!hasNonAlgebricAggregateFunctions
+  }
+
+  // We need to use sorted input if we have grouping expressions, and
+  // we cannot use the hybrid iterator or the hybrid is disabled.
+  private[this] val requiresSortedInput: Boolean = {
+    groupingExpressions.nonEmpty && !supportsHybridIterator
+  }
+
+  override def canProcessUnsafeRows: Boolean = 
!hasNonAlgebricAggregateFunctions
+
+  // If result expressions' data types are all fixed length, we generate 
unsafe rows
+  // (We have this requirement instead of check the result of 
UnsafeProjection.canSupport
+  // is because we use a mutable projection to generate the result).
+  override def outputsUnsafeRows: Boolean = {
+    // resultExpressions.map(_.dataType).forall(UnsafeRow.isFixedLength)
+    // TODO: Supports generating UnsafeRows. We can just re-enable the line 
above and fix
+    // any issue we get.
+    false
+  }
+
+  override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute)
+
+  override def requiredChildDistribution: List[Distribution] = {
+    requiredChildDistributionExpressions match {
+      case Some(exprs) if exprs.length == 0 => AllTuples :: Nil
+      case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: 
Nil
+      case None => UnspecifiedDistribution :: Nil
+    }
+  }
+
+  override def requiredChildOrdering: Seq[Seq[SortOrder]] = {
+    if (requiresSortedInput) {
+      // TODO: We should not sort the input rows if they are just in reversed 
order.
+      groupingExpressions.map(SortOrder(_, Ascending)) :: Nil
+    } else {
+      Seq.fill(children.size)(Nil)
+    }
+  }
+
+  override def outputOrdering: Seq[SortOrder] = {
+    if (requiresSortedInput) {
+      // It is possible that the child.outputOrdering starts with the required
+      // ordering expressions (e.g. we require [a] as the sort expression and 
the
+      // child's outputOrdering is [a, b]). We can only guarantee the output 
rows
+      // are sorted by values of groupingExpressions.
+      groupingExpressions.map(SortOrder(_, Ascending))
+    } else {
+      Nil
+    }
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = attachTree(this, 
"execute") {
+    child.execute().mapPartitions { iter =>
+      // Because the constructor of an aggregation iterator will read at least 
the first row,
+      // we need to get the value of iter.hasNext first.
+      val hasInput = iter.hasNext
+      val useHybridIterator =
+        hasInput &&
+          supportsHybridIterator &&
+          groupingExpressions.nonEmpty
+      if (useHybridIterator) {
+        UnsafeHybridAggregationIterator.createFromInputIterator(
+          groupingExpressions,
+          nonCompleteAggregateExpressions,
+          nonCompleteAggregateAttributes,
+          completeAggregateExpressions,
+          completeAggregateAttributes,
+          initialInputBufferOffset,
+          resultExpressions,
+          newMutableProjection _,
+          child.output,
+          iter,
+          outputsUnsafeRows)
+      } else {
+        if (!hasInput && groupingExpressions.nonEmpty) {
+          // This is a grouped aggregate and the input iterator is empty,
+          // so return an empty iterator.
+          Iterator[InternalRow]()
+        } else {
+          val outputIter = 
SortBasedAggregationIterator.createFromInputIterator(
+            groupingExpressions,
+            nonCompleteAggregateExpressions,
+            nonCompleteAggregateAttributes,
+            completeAggregateExpressions,
+            completeAggregateAttributes,
+            initialInputBufferOffset,
+            resultExpressions,
+            newMutableProjection _ ,
+            newProjection _,
+            child.output,
+            iter,
+            outputsUnsafeRows)
+          if (!hasInput && groupingExpressions.isEmpty) {
+            // There is no input and there is no grouping expressions.
+            // We need to output a single row as the output.
+            
Iterator[InternalRow](outputIter.outputForEmptyGroupingKeyWithoutInput())
+          } else {
+            outputIter
+          }
+        }
+      }
+    }
+  }
+
+  override def simpleString: String = {
+    val iterator = if (supportsHybridIterator && groupingExpressions.nonEmpty) 
{
+      classOf[UnsafeHybridAggregationIterator].getSimpleName
+    } else {
+      classOf[SortBasedAggregationIterator].getSimpleName
+    }
+
+    s"""NewAggregate with $iterator ${groupingExpressions} 
${allAggregateExpressions}"""
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala
new file mode 100644
index 0000000..abca373
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala
@@ -0,0 +1,490 @@
+/*
+ * 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.execution.aggregate
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.unsafe.KVIterator
+
+import scala.collection.mutable.ArrayBuffer
+
+/**
+ * The base class of [[SortBasedAggregationIterator]] and 
[[UnsafeHybridAggregationIterator]].
+ * It mainly contains two parts:
+ * 1. It initializes aggregate functions.
+ * 2. It creates two functions, `processRow` and `generateOutput` based on 
[[AggregateMode]] of
+ *    its aggregate functions. `processRow` is the function to handle an 
input. `generateOutput`
+ *    is used to generate result.
+ */
+abstract class AggregationIterator(
+    groupingKeyAttributes: Seq[Attribute],
+    valueAttributes: Seq[Attribute],
+    nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+    nonCompleteAggregateAttributes: Seq[Attribute],
+    completeAggregateExpressions: Seq[AggregateExpression2],
+    completeAggregateAttributes: Seq[Attribute],
+    initialInputBufferOffset: Int,
+    resultExpressions: Seq[NamedExpression],
+    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+    outputsUnsafeRows: Boolean)
+  extends Iterator[InternalRow] with Logging {
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Initializing functions.
+  ///////////////////////////////////////////////////////////////////////////
+
+  // An Seq of all AggregateExpressions.
+  // It is important that all AggregateExpressions with the mode Partial, 
PartialMerge or Final
+  // are at the beginning of the allAggregateExpressions.
+  protected val allAggregateExpressions =
+    nonCompleteAggregateExpressions ++ completeAggregateExpressions
+
+  require(
+    allAggregateExpressions.map(_.mode).distinct.length <= 2,
+    s"$allAggregateExpressions are not supported becuase they have more than 2 
distinct modes.")
+
+  /**
+   * The distinct modes of AggregateExpressions. Right now, we can handle the 
following mode:
+   *  - Partial-only: all AggregateExpressions have the mode of Partial;
+   *  - PartialMerge-only: all AggregateExpressions have the mode of 
PartialMerge);
+   *  - Final-only: all AggregateExpressions have the mode of Final;
+   *  - Final-Complete: some AggregateExpressions have the mode of Final and
+   *    others have the mode of Complete;
+   *  - Complete-only: nonCompleteAggregateExpressions is empty and we have 
AggregateExpressions
+   *    with mode Complete in completeAggregateExpressions; and
+   *  - Grouping-only: there is no AggregateExpression.
+   */
+  protected val aggregationMode: (Option[AggregateMode], 
Option[AggregateMode]) =
+    nonCompleteAggregateExpressions.map(_.mode).distinct.headOption ->
+      completeAggregateExpressions.map(_.mode).distinct.headOption
+
+  // Initialize all AggregateFunctions by binding references if necessary,
+  // and set inputBufferOffset and mutableBufferOffset.
+  protected val allAggregateFunctions: Array[AggregateFunction2] = {
+    var mutableBufferOffset = 0
+    var inputBufferOffset: Int = initialInputBufferOffset
+    val functions = new 
Array[AggregateFunction2](allAggregateExpressions.length)
+    var i = 0
+    while (i < allAggregateExpressions.length) {
+      val func = allAggregateExpressions(i).aggregateFunction
+      val funcWithBoundReferences = allAggregateExpressions(i).mode match {
+        case Partial | Complete if !func.isInstanceOf[AlgebraicAggregate] =>
+          // We need to create BoundReferences if the function is not an
+          // AlgebraicAggregate (it does not support code-gen) and the mode of
+          // this function is Partial or Complete because we will call eval of 
this
+          // function's children in the update method of this aggregate 
function.
+          // Those eval calls require BoundReferences to work.
+          BindReferences.bindReference(func, valueAttributes)
+        case _ =>
+          // We only need to set inputBufferOffset for aggregate functions 
with mode
+          // PartialMerge and Final.
+          func.withNewInputBufferOffset(inputBufferOffset)
+          inputBufferOffset += func.bufferSchema.length
+          func
+      }
+      // Set mutableBufferOffset for this function. It is important that 
setting
+      // mutableBufferOffset happens after all potential bindReference 
operations
+      // because bindReference will create a new instance of the function.
+      funcWithBoundReferences.withNewMutableBufferOffset(mutableBufferOffset)
+      mutableBufferOffset += funcWithBoundReferences.bufferSchema.length
+      functions(i) = funcWithBoundReferences
+      i += 1
+    }
+    functions
+  }
+
+  // Positions of those non-algebraic aggregate functions in 
allAggregateFunctions.
+  // For example, we have func1, func2, func3, func4 in aggregateFunctions, and
+  // func2 and func3 are non-algebraic aggregate functions.
+  // nonAlgebraicAggregateFunctionPositions will be [1, 2].
+  private[this] val allNonAlgebraicAggregateFunctionPositions: Array[Int] = {
+    val positions = new ArrayBuffer[Int]()
+    var i = 0
+    while (i < allAggregateFunctions.length) {
+      allAggregateFunctions(i) match {
+        case agg: AlgebraicAggregate =>
+        case _ => positions += i
+      }
+      i += 1
+    }
+    positions.toArray
+  }
+
+  // All AggregateFunctions functions with mode Partial, PartialMerge, or 
Final.
+  private[this] val nonCompleteAggregateFunctions: Array[AggregateFunction2] =
+    allAggregateFunctions.take(nonCompleteAggregateExpressions.length)
+
+  // All non-algebraic aggregate functions with mode Partial, PartialMerge, or 
Final.
+  private[this] val nonCompleteNonAlgebraicAggregateFunctions: 
Array[AggregateFunction2] =
+    nonCompleteAggregateFunctions.collect {
+      case func: AggregateFunction2 if !func.isInstanceOf[AlgebraicAggregate] 
=> func
+    }
+
+  // The projection used to initialize buffer values for all 
AlgebraicAggregates.
+  private[this] val algebraicInitialProjection = {
+    val initExpressions = allAggregateFunctions.flatMap {
+      case ae: AlgebraicAggregate => ae.initialValues
+      case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
+    }
+    newMutableProjection(initExpressions, Nil)()
+  }
+
+  // All non-Algebraic AggregateFunctions.
+  private[this] val allNonAlgebraicAggregateFunctions =
+    allNonAlgebraicAggregateFunctionPositions.map(allAggregateFunctions)
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Methods and fields used by sub-classes.
+  ///////////////////////////////////////////////////////////////////////////
+
+  // Initializing functions used to process a row.
+  protected val processRow: (MutableRow, InternalRow) => Unit = {
+    val rowToBeProcessed = new JoinedRow
+    val aggregationBufferSchema = 
allAggregateFunctions.flatMap(_.bufferAttributes)
+    aggregationMode match {
+      // Partial-only
+      case (Some(Partial), None) =>
+        val updateExpressions = nonCompleteAggregateFunctions.flatMap {
+          case ae: AlgebraicAggregate => ae.updateExpressions
+          case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
+        }
+        val algebraicUpdateProjection =
+          newMutableProjection(updateExpressions, aggregationBufferSchema ++ 
valueAttributes)()
+
+        (currentBuffer: MutableRow, row: InternalRow) => {
+          algebraicUpdateProjection.target(currentBuffer)
+          // Process all algebraic aggregate functions.
+          algebraicUpdateProjection(rowToBeProcessed(currentBuffer, row))
+          // Process all non-algebraic aggregate functions.
+          var i = 0
+          while (i < nonCompleteNonAlgebraicAggregateFunctions.length) {
+            nonCompleteNonAlgebraicAggregateFunctions(i).update(currentBuffer, 
row)
+            i += 1
+          }
+        }
+
+      // PartialMerge-only or Final-only
+      case (Some(PartialMerge), None) | (Some(Final), None) =>
+        val inputAggregationBufferSchema = if (initialInputBufferOffset == 0) {
+          // If initialInputBufferOffset, the input value does not contain
+          // grouping keys.
+          // This part is pretty hacky.
+          allAggregateFunctions.flatMap(_.cloneBufferAttributes).toSeq
+        } else {
+          groupingKeyAttributes ++ 
allAggregateFunctions.flatMap(_.cloneBufferAttributes)
+        }
+        // val inputAggregationBufferSchema =
+        //  groupingKeyAttributes ++
+        //    allAggregateFunctions.flatMap(_.cloneBufferAttributes)
+        val mergeExpressions = nonCompleteAggregateFunctions.flatMap {
+          case ae: AlgebraicAggregate => ae.mergeExpressions
+          case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
+        }
+        // This projection is used to merge buffer values for all 
AlgebraicAggregates.
+        val algebraicMergeProjection =
+          newMutableProjection(
+            mergeExpressions,
+            aggregationBufferSchema ++ inputAggregationBufferSchema)()
+
+        (currentBuffer: MutableRow, row: InternalRow) => {
+          // Process all algebraic aggregate functions.
+          
algebraicMergeProjection.target(currentBuffer)(rowToBeProcessed(currentBuffer, 
row))
+          // Process all non-algebraic aggregate functions.
+          var i = 0
+          while (i < nonCompleteNonAlgebraicAggregateFunctions.length) {
+            nonCompleteNonAlgebraicAggregateFunctions(i).merge(currentBuffer, 
row)
+            i += 1
+          }
+        }
+
+      // Final-Complete
+      case (Some(Final), Some(Complete)) =>
+        val completeAggregateFunctions: Array[AggregateFunction2] =
+          allAggregateFunctions.takeRight(completeAggregateExpressions.length)
+        // All non-algebraic aggregate functions with mode Complete.
+        val completeNonAlgebraicAggregateFunctions: Array[AggregateFunction2] =
+          completeAggregateFunctions.collect {
+            case func: AggregateFunction2 if 
!func.isInstanceOf[AlgebraicAggregate] => func
+          }
+
+        // The first initialInputBufferOffset values of the input aggregation 
buffer is
+        // for grouping expressions and distinct columns.
+        val groupingAttributesAndDistinctColumns = 
valueAttributes.take(initialInputBufferOffset)
+
+        val completeOffsetExpressions =
+          
Seq.fill(completeAggregateFunctions.map(_.bufferAttributes.length).sum)(NoOp)
+        // We do not touch buffer values of aggregate functions with the Final 
mode.
+        val finalOffsetExpressions =
+          
Seq.fill(nonCompleteAggregateFunctions.map(_.bufferAttributes.length).sum)(NoOp)
+
+        val mergeInputSchema =
+          aggregationBufferSchema ++
+            groupingAttributesAndDistinctColumns ++
+            nonCompleteAggregateFunctions.flatMap(_.cloneBufferAttributes)
+        val mergeExpressions =
+          nonCompleteAggregateFunctions.flatMap {
+            case ae: AlgebraicAggregate => ae.mergeExpressions
+            case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
+          } ++ completeOffsetExpressions
+        val finalAlgebraicMergeProjection =
+          newMutableProjection(mergeExpressions, mergeInputSchema)()
+
+        val updateExpressions =
+          finalOffsetExpressions ++ completeAggregateFunctions.flatMap {
+            case ae: AlgebraicAggregate => ae.updateExpressions
+            case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
+          }
+        val completeAlgebraicUpdateProjection =
+          newMutableProjection(updateExpressions, aggregationBufferSchema ++ 
valueAttributes)()
+
+        (currentBuffer: MutableRow, row: InternalRow) => {
+          val input = rowToBeProcessed(currentBuffer, row)
+          // For all aggregate functions with mode Complete, update buffers.
+          completeAlgebraicUpdateProjection.target(currentBuffer)(input)
+          var i = 0
+          while (i < completeNonAlgebraicAggregateFunctions.length) {
+            completeNonAlgebraicAggregateFunctions(i).update(currentBuffer, 
row)
+            i += 1
+          }
+
+          // For all aggregate functions with mode Final, merge buffers.
+          finalAlgebraicMergeProjection.target(currentBuffer)(input)
+          i = 0
+          while (i < nonCompleteNonAlgebraicAggregateFunctions.length) {
+            nonCompleteNonAlgebraicAggregateFunctions(i).merge(currentBuffer, 
row)
+            i += 1
+          }
+        }
+
+      // Complete-only
+      case (None, Some(Complete)) =>
+        val completeAggregateFunctions: Array[AggregateFunction2] =
+          allAggregateFunctions.takeRight(completeAggregateExpressions.length)
+        // All non-algebraic aggregate functions with mode Complete.
+        val completeNonAlgebraicAggregateFunctions: Array[AggregateFunction2] =
+          completeAggregateFunctions.collect {
+            case func: AggregateFunction2 if 
!func.isInstanceOf[AlgebraicAggregate] => func
+          }
+
+        val updateExpressions =
+          completeAggregateFunctions.flatMap {
+            case ae: AlgebraicAggregate => ae.updateExpressions
+            case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
+          }
+        val completeAlgebraicUpdateProjection =
+          newMutableProjection(updateExpressions, aggregationBufferSchema ++ 
valueAttributes)()
+
+        (currentBuffer: MutableRow, row: InternalRow) => {
+          val input = rowToBeProcessed(currentBuffer, row)
+          // For all aggregate functions with mode Complete, update buffers.
+          completeAlgebraicUpdateProjection.target(currentBuffer)(input)
+          var i = 0
+          while (i < completeNonAlgebraicAggregateFunctions.length) {
+            completeNonAlgebraicAggregateFunctions(i).update(currentBuffer, 
row)
+            i += 1
+          }
+        }
+
+      // Grouping only.
+      case (None, None) => (currentBuffer: MutableRow, row: InternalRow) => {}
+
+      case other =>
+        sys.error(
+          s"Could not evaluate ${nonCompleteAggregateExpressions} because we 
do not " +
+            s"support evaluate modes $other in this iterator.")
+    }
+  }
+
+  // Initializing the function used to generate the output row.
+  protected val generateOutput: (InternalRow, MutableRow) => InternalRow = {
+    val rowToBeEvaluated = new JoinedRow
+    val safeOutoutRow = new GenericMutableRow(resultExpressions.length)
+    val mutableOutput = if (outputsUnsafeRows) {
+      
UnsafeProjection.create(resultExpressions.map(_.dataType).toArray).apply(safeOutoutRow)
+    } else {
+      safeOutoutRow
+    }
+
+    aggregationMode match {
+      // Partial-only or PartialMerge-only: every output row is basically the 
values of
+      // the grouping expressions and the corresponding aggregation buffer.
+      case (Some(Partial), None) | (Some(PartialMerge), None) =>
+        // Because we cannot copy a joinedRow containing a UnsafeRow 
(UnsafeRow does not
+        // support generic getter), we create a mutable projection to output 
the
+        // JoinedRow(currentGroupingKey, currentBuffer)
+        val bufferSchema = 
nonCompleteAggregateFunctions.flatMap(_.bufferAttributes)
+        val resultProjection =
+          newMutableProjection(
+            groupingKeyAttributes ++ bufferSchema,
+            groupingKeyAttributes ++ bufferSchema)()
+        resultProjection.target(mutableOutput)
+
+        (currentGroupingKey: InternalRow, currentBuffer: MutableRow) => {
+          resultProjection(rowToBeEvaluated(currentGroupingKey, currentBuffer))
+          // rowToBeEvaluated(currentGroupingKey, currentBuffer)
+        }
+
+      // Final-only, Complete-only and Final-Complete: every output row 
contains values representing
+      // resultExpressions.
+      case (Some(Final), None) | (Some(Final) | None, Some(Complete)) =>
+        val bufferSchemata =
+          allAggregateFunctions.flatMap(_.bufferAttributes)
+        val evalExpressions = allAggregateFunctions.map {
+          case ae: AlgebraicAggregate => ae.evaluateExpression
+          case agg: AggregateFunction2 => NoOp
+        }
+        val algebraicEvalProjection = newMutableProjection(evalExpressions, 
bufferSchemata)()
+        val aggregateResultSchema = nonCompleteAggregateAttributes ++ 
completeAggregateAttributes
+        // TODO: Use unsafe row.
+        val aggregateResult = new 
GenericMutableRow(aggregateResultSchema.length)
+        val resultProjection =
+          newMutableProjection(
+            resultExpressions, groupingKeyAttributes ++ 
aggregateResultSchema)()
+        resultProjection.target(mutableOutput)
+
+        (currentGroupingKey: InternalRow, currentBuffer: MutableRow) => {
+          // Generate results for all algebraic aggregate functions.
+          algebraicEvalProjection.target(aggregateResult)(currentBuffer)
+          // Generate results for all non-algebraic aggregate functions.
+          var i = 0
+          while (i < allNonAlgebraicAggregateFunctions.length) {
+            aggregateResult.update(
+              allNonAlgebraicAggregateFunctionPositions(i),
+              allNonAlgebraicAggregateFunctions(i).eval(currentBuffer))
+            i += 1
+          }
+          resultProjection(rowToBeEvaluated(currentGroupingKey, 
aggregateResult))
+        }
+
+      // Grouping-only: we only output values of grouping expressions.
+      case (None, None) =>
+        val resultProjection =
+          newMutableProjection(resultExpressions, groupingKeyAttributes)()
+        resultProjection.target(mutableOutput)
+
+        (currentGroupingKey: InternalRow, currentBuffer: MutableRow) => {
+          resultProjection(currentGroupingKey)
+        }
+
+      case other =>
+        sys.error(
+          s"Could not evaluate ${nonCompleteAggregateExpressions} because we 
do not " +
+            s"support evaluate modes $other in this iterator.")
+    }
+  }
+
+  /** Initializes buffer values for all aggregate functions. */
+  protected def initializeBuffer(buffer: MutableRow): Unit = {
+    algebraicInitialProjection.target(buffer)(EmptyRow)
+    var i = 0
+    while (i < allNonAlgebraicAggregateFunctions.length) {
+      allNonAlgebraicAggregateFunctions(i).initialize(buffer)
+      i += 1
+    }
+  }
+
+  /**
+   * Creates a new aggregation buffer and initializes buffer values
+   * for all aggregate functions.
+   */
+  protected def newBuffer: MutableRow
+}
+
+object AggregationIterator {
+  def kvIterator(
+    groupingExpressions: Seq[NamedExpression],
+    newProjection: (Seq[Expression], Seq[Attribute]) => Projection,
+    inputAttributes: Seq[Attribute],
+    inputIter: Iterator[InternalRow]): KVIterator[InternalRow, InternalRow] = {
+    new KVIterator[InternalRow, InternalRow] {
+      private[this] val groupingKeyGenerator = 
newProjection(groupingExpressions, inputAttributes)
+
+      private[this] var groupingKey: InternalRow = _
+
+      private[this] var value: InternalRow = _
+
+      override def next(): Boolean = {
+        if (inputIter.hasNext) {
+          // Read the next input row.
+          val inputRow = inputIter.next()
+          // Get groupingKey based on groupingExpressions.
+          groupingKey = groupingKeyGenerator(inputRow)
+          // The value is the inputRow.
+          value = inputRow
+          true
+        } else {
+          false
+        }
+      }
+
+      override def getKey(): InternalRow = {
+        groupingKey
+      }
+
+      override def getValue(): InternalRow = {
+        value
+      }
+
+      override def close(): Unit = {
+        // Do nothing
+      }
+    }
+  }
+
+  def unsafeKVIterator(
+      groupingExpressions: Seq[NamedExpression],
+      inputAttributes: Seq[Attribute],
+      inputIter: Iterator[InternalRow]): KVIterator[UnsafeRow, InternalRow] = {
+    new KVIterator[UnsafeRow, InternalRow] {
+      private[this] val groupingKeyGenerator =
+        UnsafeProjection.create(groupingExpressions, inputAttributes)
+
+      private[this] var groupingKey: UnsafeRow = _
+
+      private[this] var value: InternalRow = _
+
+      override def next(): Boolean = {
+        if (inputIter.hasNext) {
+          // Read the next input row.
+          val inputRow = inputIter.next()
+          // Get groupingKey based on groupingExpressions.
+          groupingKey = groupingKeyGenerator.apply(inputRow)
+          // The value is the inputRow.
+          value = inputRow
+          true
+        } else {
+          false
+        }
+      }
+
+      override def getKey(): UnsafeRow = {
+        groupingKey
+      }
+
+      override def getValue(): InternalRow = {
+        value
+      }
+
+      override def close(): Unit = {
+        // Do nothing
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala
new file mode 100644
index 0000000..78bcee1
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala
@@ -0,0 +1,236 @@
+/*
+ * 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.execution.aggregate
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression2, 
AggregateFunction2}
+import org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.unsafe.KVIterator
+
+/**
+ * An iterator used to evaluate [[AggregateFunction2]]. It assumes the input 
rows have been
+ * sorted by values of [[groupingKeyAttributes]].
+ */
+class SortBasedAggregationIterator(
+    groupingKeyAttributes: Seq[Attribute],
+    valueAttributes: Seq[Attribute],
+    inputKVIterator: KVIterator[InternalRow, InternalRow],
+    nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+    nonCompleteAggregateAttributes: Seq[Attribute],
+    completeAggregateExpressions: Seq[AggregateExpression2],
+    completeAggregateAttributes: Seq[Attribute],
+    initialInputBufferOffset: Int,
+    resultExpressions: Seq[NamedExpression],
+    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+    outputsUnsafeRows: Boolean)
+  extends AggregationIterator(
+    groupingKeyAttributes,
+    valueAttributes,
+    nonCompleteAggregateExpressions,
+    nonCompleteAggregateAttributes,
+    completeAggregateExpressions,
+    completeAggregateAttributes,
+    initialInputBufferOffset,
+    resultExpressions,
+    newMutableProjection,
+    outputsUnsafeRows) {
+
+  override protected def newBuffer: MutableRow = {
+    val bufferSchema = allAggregateFunctions.flatMap(_.bufferAttributes)
+    val bufferRowSize: Int = bufferSchema.length
+
+    val genericMutableBuffer = new GenericMutableRow(bufferRowSize)
+    val useUnsafeBuffer = 
bufferSchema.map(_.dataType).forall(UnsafeRow.isFixedLength)
+
+    val buffer = if (useUnsafeBuffer) {
+      val unsafeProjection =
+        UnsafeProjection.create(bufferSchema.map(_.dataType))
+      unsafeProjection.apply(genericMutableBuffer)
+    } else {
+      genericMutableBuffer
+    }
+    initializeBuffer(buffer)
+    buffer
+  }
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Mutable states for sort based aggregation.
+  ///////////////////////////////////////////////////////////////////////////
+
+  // The partition key of the current partition.
+  private[this] var currentGroupingKey: InternalRow = _
+
+  // The partition key of next partition.
+  private[this] var nextGroupingKey: InternalRow = _
+
+  // The first row of next partition.
+  private[this] var firstRowInNextGroup: InternalRow = _
+
+  // Indicates if we has new group of rows from the sorted input iterator
+  private[this] var sortedInputHasNewGroup: Boolean = false
+
+  // The aggregation buffer used by the sort-based aggregation.
+  private[this] val sortBasedAggregationBuffer: MutableRow = newBuffer
+
+  /** Processes rows in the current group. It will stop when it find a new 
group. */
+  protected def processCurrentSortedGroup(): Unit = {
+    currentGroupingKey = nextGroupingKey
+    // Now, we will start to find all rows belonging to this group.
+    // We create a variable to track if we see the next group.
+    var findNextPartition = false
+    // firstRowInNextGroup is the first row of this group. We first process it.
+    processRow(sortBasedAggregationBuffer, firstRowInNextGroup)
+
+    // The search will stop when we see the next group or there is no
+    // input row left in the iter.
+    var hasNext = inputKVIterator.next()
+    while (!findNextPartition && hasNext) {
+      // Get the grouping key.
+      val groupingKey = inputKVIterator.getKey
+      val currentRow = inputKVIterator.getValue
+
+      // Check if the current row belongs the current input row.
+      if (currentGroupingKey == groupingKey) {
+        processRow(sortBasedAggregationBuffer, currentRow)
+
+        hasNext = inputKVIterator.next()
+      } else {
+        // We find a new group.
+        findNextPartition = true
+        nextGroupingKey = groupingKey.copy()
+        firstRowInNextGroup = currentRow.copy()
+      }
+    }
+    // We have not seen a new group. It means that there is no new row in the 
input
+    // iter. The current group is the last group of the iter.
+    if (!findNextPartition) {
+      sortedInputHasNewGroup = false
+    }
+  }
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Iterator's public methods
+  ///////////////////////////////////////////////////////////////////////////
+
+  override final def hasNext: Boolean = sortedInputHasNewGroup
+
+  override final def next(): InternalRow = {
+    if (hasNext) {
+      // Process the current group.
+      processCurrentSortedGroup()
+      // Generate output row for the current group.
+      val outputRow = generateOutput(currentGroupingKey, 
sortBasedAggregationBuffer)
+      // Initialize buffer values for the next group.
+      initializeBuffer(sortBasedAggregationBuffer)
+
+      outputRow
+    } else {
+      // no more result
+      throw new NoSuchElementException
+    }
+  }
+
+  protected def initialize(): Unit = {
+    if (inputKVIterator.next()) {
+      initializeBuffer(sortBasedAggregationBuffer)
+
+      nextGroupingKey = inputKVIterator.getKey().copy()
+      firstRowInNextGroup = inputKVIterator.getValue().copy()
+
+      sortedInputHasNewGroup = true
+    } else {
+      // This inputIter is empty.
+      sortedInputHasNewGroup = false
+    }
+  }
+
+  initialize()
+
+  def outputForEmptyGroupingKeyWithoutInput(): InternalRow = {
+    initializeBuffer(sortBasedAggregationBuffer)
+    generateOutput(new GenericInternalRow(0), sortBasedAggregationBuffer)
+  }
+}
+
+object SortBasedAggregationIterator {
+  // scalastyle:off
+  def createFromInputIterator(
+      groupingExprs: Seq[NamedExpression],
+      nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+      nonCompleteAggregateAttributes: Seq[Attribute],
+      completeAggregateExpressions: Seq[AggregateExpression2],
+      completeAggregateAttributes: Seq[Attribute],
+      initialInputBufferOffset: Int,
+      resultExpressions: Seq[NamedExpression],
+      newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+      newProjection: (Seq[Expression], Seq[Attribute]) => Projection,
+      inputAttributes: Seq[Attribute],
+      inputIter: Iterator[InternalRow],
+      outputsUnsafeRows: Boolean): SortBasedAggregationIterator = {
+    val kvIterator = if (UnsafeProjection.canSupport(groupingExprs)) {
+      AggregationIterator.unsafeKVIterator(
+        groupingExprs,
+        inputAttributes,
+        inputIter).asInstanceOf[KVIterator[InternalRow, InternalRow]]
+    } else {
+      AggregationIterator.kvIterator(groupingExprs, newProjection, 
inputAttributes, inputIter)
+    }
+
+    new SortBasedAggregationIterator(
+      groupingExprs.map(_.toAttribute),
+      inputAttributes,
+      kvIterator,
+      nonCompleteAggregateExpressions,
+      nonCompleteAggregateAttributes,
+      completeAggregateExpressions,
+      completeAggregateAttributes,
+      initialInputBufferOffset,
+      resultExpressions,
+      newMutableProjection,
+      outputsUnsafeRows)
+  }
+
+  def createFromKVIterator(
+      groupingKeyAttributes: Seq[Attribute],
+      valueAttributes: Seq[Attribute],
+      inputKVIterator: KVIterator[InternalRow, InternalRow],
+      nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+      nonCompleteAggregateAttributes: Seq[Attribute],
+      completeAggregateExpressions: Seq[AggregateExpression2],
+      completeAggregateAttributes: Seq[Attribute],
+      initialInputBufferOffset: Int,
+      resultExpressions: Seq[NamedExpression],
+      newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+      outputsUnsafeRows: Boolean): SortBasedAggregationIterator = {
+    new SortBasedAggregationIterator(
+      groupingKeyAttributes,
+      valueAttributes,
+      inputKVIterator,
+      nonCompleteAggregateExpressions,
+      nonCompleteAggregateAttributes,
+      completeAggregateExpressions,
+      completeAggregateAttributes,
+      initialInputBufferOffset,
+      resultExpressions,
+      newMutableProjection,
+      outputsUnsafeRows)
+  }
+  // scalastyle:on
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala
new file mode 100644
index 0000000..37d34eb
--- /dev/null
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala
@@ -0,0 +1,398 @@
+/*
+ * 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.execution.aggregate
+
+import org.apache.spark.sql.execution.{UnsafeKeyValueSorter, 
UnsafeFixedWidthAggregationMap}
+import org.apache.spark.unsafe.KVIterator
+import org.apache.spark.{SparkEnv, TaskContext}
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.types.StructType
+
+/**
+ * An iterator used to evaluate [[AggregateFunction2]].
+ * It first tries to use in-memory hash-based aggregation. If we cannot 
allocate more
+ * space for the hash map, we spill the sorted map entries, free the map, and 
then
+ * switch to sort-based aggregation.
+ */
+class UnsafeHybridAggregationIterator(
+    groupingKeyAttributes: Seq[Attribute],
+    valueAttributes: Seq[Attribute],
+    inputKVIterator: KVIterator[UnsafeRow, InternalRow],
+    nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+    nonCompleteAggregateAttributes: Seq[Attribute],
+    completeAggregateExpressions: Seq[AggregateExpression2],
+    completeAggregateAttributes: Seq[Attribute],
+    initialInputBufferOffset: Int,
+    resultExpressions: Seq[NamedExpression],
+    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+    outputsUnsafeRows: Boolean)
+  extends AggregationIterator(
+    groupingKeyAttributes,
+    valueAttributes,
+    nonCompleteAggregateExpressions,
+    nonCompleteAggregateAttributes,
+    completeAggregateExpressions,
+    completeAggregateAttributes,
+    initialInputBufferOffset,
+    resultExpressions,
+    newMutableProjection,
+    outputsUnsafeRows) {
+
+  require(groupingKeyAttributes.nonEmpty)
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Unsafe Aggregation buffers
+  ///////////////////////////////////////////////////////////////////////////
+
+  // This is the Unsafe Aggregation Map used to store all buffers.
+  private[this] val buffers = new UnsafeFixedWidthAggregationMap(
+    newBuffer,
+    
StructType.fromAttributes(allAggregateFunctions.flatMap(_.bufferAttributes)),
+    StructType.fromAttributes(groupingKeyAttributes),
+    TaskContext.get.taskMemoryManager(),
+    SparkEnv.get.shuffleMemoryManager,
+    1024 * 16, // initial capacity
+    SparkEnv.get.conf.getSizeAsBytes("spark.buffer.pageSize", "64m"),
+    false // disable tracking of performance metrics
+  )
+
+  override protected def newBuffer: UnsafeRow = {
+    val bufferSchema = allAggregateFunctions.flatMap(_.bufferAttributes)
+    val bufferRowSize: Int = bufferSchema.length
+
+    val genericMutableBuffer = new GenericMutableRow(bufferRowSize)
+    val unsafeProjection =
+      UnsafeProjection.create(bufferSchema.map(_.dataType))
+    val buffer = unsafeProjection.apply(genericMutableBuffer)
+    initializeBuffer(buffer)
+    buffer
+  }
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Methods and variables related to switching to sort-based aggregation
+  ///////////////////////////////////////////////////////////////////////////
+  private[this] var sortBased = false
+
+  private[this] var sortBasedAggregationIterator: SortBasedAggregationIterator 
= _
+
+  // The value part of the input KV iterator is used to store original input 
values of
+  // aggregate functions, we need to convert them to aggregation buffers.
+  private def processOriginalInput(
+      firstKey: UnsafeRow,
+      firstValue: InternalRow): KVIterator[UnsafeRow, UnsafeRow] = {
+    new KVIterator[UnsafeRow, UnsafeRow] {
+      private[this] var isFirstRow = true
+
+      private[this] var groupingKey: UnsafeRow = _
+
+      private[this] val buffer: UnsafeRow = newBuffer
+
+      override def next(): Boolean = {
+        initializeBuffer(buffer)
+        if (isFirstRow) {
+          isFirstRow = false
+          groupingKey = firstKey
+          processRow(buffer, firstValue)
+
+          true
+        } else if (inputKVIterator.next()) {
+          groupingKey = inputKVIterator.getKey()
+          val value = inputKVIterator.getValue()
+          processRow(buffer, value)
+
+          true
+        } else {
+          false
+        }
+      }
+
+      override def getKey(): UnsafeRow = {
+        groupingKey
+      }
+
+      override def getValue(): UnsafeRow = {
+        buffer
+      }
+
+      override def close(): Unit = {
+        // Do nothing.
+      }
+    }
+  }
+
+  // The value of the input KV Iterator has the format of groupingExprs + 
aggregation buffer.
+  // We need to project the aggregation buffer out.
+  private def projectInputBufferToUnsafe(
+      firstKey: UnsafeRow,
+      firstValue: InternalRow): KVIterator[UnsafeRow, UnsafeRow] = {
+    new KVIterator[UnsafeRow, UnsafeRow] {
+      private[this] var isFirstRow = true
+
+      private[this] var groupingKey: UnsafeRow = _
+
+      private[this] val bufferSchema = 
allAggregateFunctions.flatMap(_.bufferAttributes)
+
+      private[this] val value: UnsafeRow = {
+        val genericMutableRow = new GenericMutableRow(bufferSchema.length)
+        
UnsafeProjection.create(bufferSchema.map(_.dataType)).apply(genericMutableRow)
+      }
+
+      private[this] val projectInputBuffer = {
+        newMutableProjection(bufferSchema, valueAttributes)().target(value)
+      }
+
+      override def next(): Boolean = {
+        if (isFirstRow) {
+          isFirstRow = false
+          groupingKey = firstKey
+          projectInputBuffer(firstValue)
+
+          true
+        } else if (inputKVIterator.next()) {
+          groupingKey = inputKVIterator.getKey()
+          projectInputBuffer(inputKVIterator.getValue())
+
+          true
+        } else {
+          false
+        }
+      }
+
+      override def getKey(): UnsafeRow = {
+        groupingKey
+      }
+
+      override def getValue(): UnsafeRow = {
+        value
+      }
+
+      override def close(): Unit = {
+        // Do nothing.
+      }
+    }
+  }
+
+  /**
+   * We need to fall back to sort based aggregation because we do not have 
enough memory
+   * for our in-memory hash map (i.e. `buffers`).
+   */
+  private def switchToSortBasedAggregation(
+      currentGroupingKey: UnsafeRow,
+      currentRow: InternalRow): Unit = {
+    logInfo("falling back to sort based aggregation.")
+
+    // Step 1: Get the ExternalSorter containing entries of the map.
+    val externalSorter = buffers.destructAndCreateExternalSorter()
+
+    // Step 2: Free the memory used by the map.
+    buffers.free()
+
+    // Step 3: If we have aggregate function with mode Partial or Complete,
+    // we need to process them to get aggregation buffer.
+    // So, later in the sort-based aggregation iterator, we can do merge.
+    // If aggregate functions are with mode Final and PartialMerge,
+    // we just need to project the aggregation buffer from the input.
+    val needsProcess = aggregationMode match {
+      case (Some(Partial), None) => true
+      case (None, Some(Complete)) => true
+      case (Some(Final), Some(Complete)) => true
+      case _ => false
+    }
+
+    val processedIterator = if (needsProcess) {
+      processOriginalInput(currentGroupingKey, currentRow)
+    } else {
+      // The input value's format is groupingExprs + buffer.
+      // We need to project the buffer part out.
+      projectInputBufferToUnsafe(currentGroupingKey, currentRow)
+    }
+
+    // Step 4: Redirect processedIterator to externalSorter.
+    while (processedIterator.next()) {
+      externalSorter.insertKV(processedIterator.getKey(), 
processedIterator.getValue())
+    }
+
+    // Step 5: Get the sorted iterator from the externalSorter.
+    val sortedKVIterator: KVIterator[UnsafeRow, UnsafeRow] = 
externalSorter.sortedIterator()
+
+    // Step 6: We now create a SortBasedAggregationIterator based on 
sortedKVIterator.
+    // For a aggregate function with mode Partial, its mode in the 
SortBasedAggregationIterator
+    // will be PartialMerge. For a aggregate function with mode Complete,
+    // its mode in the SortBasedAggregationIterator will be Final.
+    val newNonCompleteAggregateExpressions = allAggregateExpressions.map {
+        case AggregateExpression2(func, Partial, isDistinct) =>
+          AggregateExpression2(func, PartialMerge, isDistinct)
+        case AggregateExpression2(func, Complete, isDistinct) =>
+          AggregateExpression2(func, Final, isDistinct)
+        case other => other
+      }
+    val newNonCompleteAggregateAttributes =
+      nonCompleteAggregateAttributes ++ completeAggregateAttributes
+
+    val newValueAttributes =
+      
allAggregateExpressions.flatMap(_.aggregateFunction.cloneBufferAttributes)
+
+    sortBasedAggregationIterator = 
SortBasedAggregationIterator.createFromKVIterator(
+      groupingKeyAttributes = groupingKeyAttributes,
+      valueAttributes = newValueAttributes,
+      inputKVIterator = sortedKVIterator.asInstanceOf[KVIterator[InternalRow, 
InternalRow]],
+      nonCompleteAggregateExpressions = newNonCompleteAggregateExpressions,
+      nonCompleteAggregateAttributes = newNonCompleteAggregateAttributes,
+      completeAggregateExpressions = Nil,
+      completeAggregateAttributes = Nil,
+      initialInputBufferOffset = 0,
+      resultExpressions = resultExpressions,
+      newMutableProjection = newMutableProjection,
+      outputsUnsafeRows = outputsUnsafeRows)
+  }
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Methods used to initialize this iterator.
+  ///////////////////////////////////////////////////////////////////////////
+
+  /** Starts to read input rows and falls back to sort-based aggregation if 
necessary. */
+  protected def initialize(): Unit = {
+    var hasNext = inputKVIterator.next()
+    while (!sortBased && hasNext) {
+      val groupingKey = inputKVIterator.getKey()
+      val currentRow = inputKVIterator.getValue()
+      val buffer = buffers.getAggregationBuffer(groupingKey)
+      if (buffer == null) {
+        // buffer == null means that we could not allocate more memory.
+        // Now, we need to spill the map and switch to sort-based aggregation.
+        switchToSortBasedAggregation(groupingKey, currentRow)
+        sortBased = true
+      } else {
+        processRow(buffer, currentRow)
+        hasNext = inputKVIterator.next()
+      }
+    }
+  }
+
+  // This is the starting point of this iterator.
+  initialize()
+
+  // Creates the iterator for the Hash Aggregation Map after we have populated
+  // contents of that map.
+  private[this] val aggregationBufferMapIterator = buffers.iterator()
+
+  private[this] var _mapIteratorHasNext = false
+
+  // Pre-load the first key-value pair from the map to make hasNext idempotent.
+  if (!sortBased) {
+    _mapIteratorHasNext = aggregationBufferMapIterator.next()
+    // If the map is empty, we just free it.
+    if (!_mapIteratorHasNext) {
+      buffers.free()
+    }
+  }
+
+  ///////////////////////////////////////////////////////////////////////////
+  // Iterator's public methods
+  ///////////////////////////////////////////////////////////////////////////
+
+  override final def hasNext: Boolean = {
+    (sortBased && sortBasedAggregationIterator.hasNext) || (!sortBased && 
_mapIteratorHasNext)
+  }
+
+
+  override final def next(): InternalRow = {
+    if (hasNext) {
+      if (sortBased) {
+        sortBasedAggregationIterator.next()
+      } else {
+        // We did not fall back to the sort-based aggregation.
+        val result =
+          generateOutput(
+            aggregationBufferMapIterator.getKey,
+            aggregationBufferMapIterator.getValue)
+        // Pre-load next key-value pair form aggregationBufferMapIterator.
+        _mapIteratorHasNext = aggregationBufferMapIterator.next()
+
+        if (!_mapIteratorHasNext) {
+          val resultCopy = result.copy()
+          buffers.free()
+          resultCopy
+        } else {
+          result
+        }
+      }
+    } else {
+      // no more result
+      throw new NoSuchElementException
+    }
+  }
+}
+
+object UnsafeHybridAggregationIterator {
+  // scalastyle:off
+  def createFromInputIterator(
+      groupingExprs: Seq[NamedExpression],
+      nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+      nonCompleteAggregateAttributes: Seq[Attribute],
+      completeAggregateExpressions: Seq[AggregateExpression2],
+      completeAggregateAttributes: Seq[Attribute],
+      initialInputBufferOffset: Int,
+      resultExpressions: Seq[NamedExpression],
+      newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+      inputAttributes: Seq[Attribute],
+      inputIter: Iterator[InternalRow],
+      outputsUnsafeRows: Boolean): UnsafeHybridAggregationIterator = {
+    new UnsafeHybridAggregationIterator(
+      groupingExprs.map(_.toAttribute),
+      inputAttributes,
+      AggregationIterator.unsafeKVIterator(groupingExprs, inputAttributes, 
inputIter),
+      nonCompleteAggregateExpressions,
+      nonCompleteAggregateAttributes,
+      completeAggregateExpressions,
+      completeAggregateAttributes,
+      initialInputBufferOffset,
+      resultExpressions,
+      newMutableProjection,
+      outputsUnsafeRows)
+  }
+
+  def createFromKVIterator(
+      groupingKeyAttributes: Seq[Attribute],
+      valueAttributes: Seq[Attribute],
+      inputKVIterator: KVIterator[UnsafeRow, InternalRow],
+      nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+      nonCompleteAggregateAttributes: Seq[Attribute],
+      completeAggregateExpressions: Seq[AggregateExpression2],
+      completeAggregateAttributes: Seq[Attribute],
+      initialInputBufferOffset: Int,
+      resultExpressions: Seq[NamedExpression],
+      newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
+      outputsUnsafeRows: Boolean): UnsafeHybridAggregationIterator = {
+    new UnsafeHybridAggregationIterator(
+      groupingKeyAttributes,
+      valueAttributes,
+      inputKVIterator,
+      nonCompleteAggregateExpressions,
+      nonCompleteAggregateAttributes,
+      completeAggregateExpressions,
+      completeAggregateAttributes,
+      initialInputBufferOffset,
+      resultExpressions,
+      newMutableProjection,
+      outputsUnsafeRows)
+  }
+  // scalastyle:on
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala
deleted file mode 100644
index 98538c4..0000000
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala
+++ /dev/null
@@ -1,175 +0,0 @@
-/*
- * 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.execution.aggregate
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, 
ClusteredDistribution, Distribution, UnspecifiedDistribution}
-import org.apache.spark.sql.execution.{SparkPlan, UnaryNode}
-
-case class Aggregate2Sort(
-    requiredChildDistributionExpressions: Option[Seq[Expression]],
-    groupingExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[AggregateExpression2],
-    aggregateAttributes: Seq[Attribute],
-    resultExpressions: Seq[NamedExpression],
-    child: SparkPlan)
-  extends UnaryNode {
-
-  override def canProcessUnsafeRows: Boolean = true
-
-  override def references: AttributeSet = {
-    val referencesInResults =
-      AttributeSet(resultExpressions.flatMap(_.references)) -- 
AttributeSet(aggregateAttributes)
-
-    AttributeSet(
-      groupingExpressions.flatMap(_.references) ++
-      aggregateExpressions.flatMap(_.references) ++
-      referencesInResults)
-  }
-
-  override def requiredChildDistribution: List[Distribution] = {
-    requiredChildDistributionExpressions match {
-      case Some(exprs) if exprs.length == 0 => AllTuples :: Nil
-      case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: 
Nil
-      case None => UnspecifiedDistribution :: Nil
-    }
-  }
-
-  override def requiredChildOrdering: Seq[Seq[SortOrder]] = {
-    // TODO: We should not sort the input rows if they are just in reversed 
order.
-    groupingExpressions.map(SortOrder(_, Ascending)) :: Nil
-  }
-
-  override def outputOrdering: Seq[SortOrder] = {
-    // It is possible that the child.outputOrdering starts with the required
-    // ordering expressions (e.g. we require [a] as the sort expression and the
-    // child's outputOrdering is [a, b]). We can only guarantee the output rows
-    // are sorted by values of groupingExpressions.
-    groupingExpressions.map(SortOrder(_, Ascending))
-  }
-
-  override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute)
-
-  protected override def doExecute(): RDD[InternalRow] = attachTree(this, 
"execute") {
-    child.execute().mapPartitions { iter =>
-      if (aggregateExpressions.length == 0) {
-        new FinalSortAggregationIterator(
-          groupingExpressions,
-          Nil,
-          Nil,
-          resultExpressions,
-          newMutableProjection,
-          child.output,
-          iter)
-      } else {
-        val aggregationIterator: SortAggregationIterator = {
-          aggregateExpressions.map(_.mode).distinct.toList match {
-            case Partial :: Nil =>
-              new PartialSortAggregationIterator(
-                groupingExpressions,
-                aggregateExpressions,
-                newMutableProjection,
-                child.output,
-                iter)
-            case PartialMerge :: Nil =>
-              new PartialMergeSortAggregationIterator(
-                groupingExpressions,
-                aggregateExpressions,
-                newMutableProjection,
-                child.output,
-                iter)
-            case Final :: Nil =>
-              new FinalSortAggregationIterator(
-                groupingExpressions,
-                aggregateExpressions,
-                aggregateAttributes,
-                resultExpressions,
-                newMutableProjection,
-                child.output,
-                iter)
-            case other =>
-              sys.error(
-                s"Could not evaluate ${aggregateExpressions} because we do not 
support evaluate " +
-                  s"modes $other in this operator.")
-          }
-        }
-
-        aggregationIterator
-      }
-    }
-  }
-}
-
-case class FinalAndCompleteAggregate2Sort(
-    previousGroupingExpressions: Seq[NamedExpression],
-    groupingExpressions: Seq[NamedExpression],
-    finalAggregateExpressions: Seq[AggregateExpression2],
-    finalAggregateAttributes: Seq[Attribute],
-    completeAggregateExpressions: Seq[AggregateExpression2],
-    completeAggregateAttributes: Seq[Attribute],
-    resultExpressions: Seq[NamedExpression],
-    child: SparkPlan)
-  extends UnaryNode {
-  override def references: AttributeSet = {
-    val referencesInResults =
-      AttributeSet(resultExpressions.flatMap(_.references)) --
-        AttributeSet(finalAggregateExpressions) --
-        AttributeSet(completeAggregateExpressions)
-
-    AttributeSet(
-      groupingExpressions.flatMap(_.references) ++
-        finalAggregateExpressions.flatMap(_.references) ++
-        completeAggregateExpressions.flatMap(_.references) ++
-        referencesInResults)
-  }
-
-  override def requiredChildDistribution: List[Distribution] = {
-    if (groupingExpressions.isEmpty) {
-      AllTuples :: Nil
-    } else {
-      ClusteredDistribution(groupingExpressions) :: Nil
-    }
-  }
-
-  override def requiredChildOrdering: Seq[Seq[SortOrder]] =
-    groupingExpressions.map(SortOrder(_, Ascending)) :: Nil
-
-  override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute)
-
-  protected override def doExecute(): RDD[InternalRow] = attachTree(this, 
"execute") {
-    child.execute().mapPartitions { iter =>
-
-      new FinalAndCompleteSortAggregationIterator(
-        previousGroupingExpressions.length,
-        groupingExpressions,
-        finalAggregateExpressions,
-        finalAggregateAttributes,
-        completeAggregateExpressions,
-        completeAggregateAttributes,
-        resultExpressions,
-        newMutableProjection,
-        child.output,
-        iter)
-    }
-  }
-
-}


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to