Repository: spark
Updated Branches:
  refs/heads/master 98d6d9c7a -> 1ebd41b14


http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/sortBasedIterators.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/sortBasedIterators.scala
 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/sortBasedIterators.scala
deleted file mode 100644
index 2ca0cb8..0000000
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/sortBasedIterators.scala
+++ /dev/null
@@ -1,664 +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.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.types.NullType
-
-import scala.collection.mutable.ArrayBuffer
-
-/**
- * An iterator used to evaluate aggregate functions. It assumes that input rows
- * are already grouped by values of `groupingExpressions`.
- */
-private[sql] abstract class SortAggregationIterator(
-    groupingExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[AggregateExpression2],
-    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
-    inputAttributes: Seq[Attribute],
-    inputIter: Iterator[InternalRow])
-  extends Iterator[InternalRow] {
-
-  ///////////////////////////////////////////////////////////////////////////
-  // Static fields for this iterator
-  ///////////////////////////////////////////////////////////////////////////
-
-  protected val aggregateFunctions: Array[AggregateFunction2] = {
-    var mutableBufferOffset = 0
-    var inputBufferOffset: Int = initialInputBufferOffset
-    val functions = new Array[AggregateFunction2](aggregateExpressions.length)
-    var i = 0
-    while (i < aggregateExpressions.length) {
-      val func = aggregateExpressions(i).aggregateFunction
-      val funcWithBoundReferences = aggregateExpressions(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, inputAttributes)
-        case _ =>
-          // We only need to set inputBufferOffset for aggregate functions 
with mode
-          // PartialMerge and Final.
-          func.inputBufferOffset = 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.mutableBufferOffset = mutableBufferOffset
-      mutableBufferOffset += funcWithBoundReferences.bufferSchema.length
-      functions(i) = funcWithBoundReferences
-      i += 1
-    }
-    functions
-  }
-
-  // Positions of those non-algebraic aggregate functions in 
aggregateFunctions.
-  // For example, we have func1, func2, func3, func4 in aggregateFunctions, and
-  // func2 and func3 are non-algebraic aggregate functions.
-  // nonAlgebraicAggregateFunctionPositions will be [1, 2].
-  protected val nonAlgebraicAggregateFunctionPositions: Array[Int] = {
-    val positions = new ArrayBuffer[Int]()
-    var i = 0
-    while (i < aggregateFunctions.length) {
-      aggregateFunctions(i) match {
-        case agg: AlgebraicAggregate =>
-        case _ => positions += i
-      }
-      i += 1
-    }
-    positions.toArray
-  }
-
-  // All non-algebraic aggregate functions.
-  protected val nonAlgebraicAggregateFunctions: Array[AggregateFunction2] =
-    nonAlgebraicAggregateFunctionPositions.map(aggregateFunctions)
-
-  // This is used to project expressions for the grouping expressions.
-  protected val groupGenerator =
-    newMutableProjection(groupingExpressions, inputAttributes)()
-
-  // The underlying buffer shared by all aggregate functions.
-  protected val buffer: MutableRow = {
-    // The number of elements of the underlying buffer of this operator.
-    // All aggregate functions are sharing this underlying buffer and they 
find their
-    // buffer values through bufferOffset.
-    // var size = 0
-    // var i = 0
-    // while (i < aggregateFunctions.length) {
-    //  size += aggregateFunctions(i).bufferSchema.length
-    //  i += 1
-    // }
-    new GenericMutableRow(aggregateFunctions.map(_.bufferSchema.length).sum)
-  }
-
-  protected val joinedRow = new JoinedRow
-
-  // This projection is used to initialize buffer values for all 
AlgebraicAggregates.
-  protected val algebraicInitialProjection = {
-    val initExpressions = aggregateFunctions.flatMap {
-      case ae: AlgebraicAggregate => ae.initialValues
-      case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
-    }
-
-    newMutableProjection(initExpressions, Nil)().target(buffer)
-  }
-
-  ///////////////////////////////////////////////////////////////////////////
-  // Mutable states
-  ///////////////////////////////////////////////////////////////////////////
-
-  // The partition key of the current partition.
-  protected var currentGroupingKey: InternalRow = _
-  // The partition key of next partition.
-  protected var nextGroupingKey: InternalRow = _
-  // The first row of next partition.
-  protected var firstRowInNextGroup: InternalRow = _
-  // Indicates if we has new group of rows to process.
-  protected var hasNewGroup: Boolean = true
-
-  /** Initializes buffer values for all aggregate functions. */
-  protected def initializeBuffer(): Unit = {
-    algebraicInitialProjection(EmptyRow)
-    var i = 0
-    while (i < nonAlgebraicAggregateFunctions.length) {
-      nonAlgebraicAggregateFunctions(i).initialize(buffer)
-      i += 1
-    }
-  }
-
-  protected def initialize(): Unit = {
-    if (inputIter.hasNext) {
-      initializeBuffer()
-      val currentRow = inputIter.next().copy()
-      // partitionGenerator is a mutable projection. Since we need to track 
nextGroupingKey,
-      // we are making a copy at here.
-      nextGroupingKey = groupGenerator(currentRow).copy()
-      firstRowInNextGroup = currentRow
-    } else {
-      // This iter is an empty one.
-      hasNewGroup = false
-    }
-  }
-
-  ///////////////////////////////////////////////////////////////////////////
-  // Private methods
-  ///////////////////////////////////////////////////////////////////////////
-
-  /** Processes rows in the current group. It will stop when it find a new 
group. */
-  private def processCurrentGroup(): 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(firstRowInNextGroup)
-    // The search will stop when we see the next group or there is no
-    // input row left in the iter.
-    while (inputIter.hasNext && !findNextPartition) {
-      val currentRow = inputIter.next()
-      // Get the grouping key based on the grouping expressions.
-      // For the below compare method, we do not need to make a copy of 
groupingKey.
-      val groupingKey = groupGenerator(currentRow)
-      // Check if the current row belongs the current input row.
-      if (currentGroupingKey == groupingKey) {
-        processRow(currentRow)
-      } 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) {
-      hasNewGroup = false
-    }
-  }
-
-  ///////////////////////////////////////////////////////////////////////////
-  // Public methods
-  ///////////////////////////////////////////////////////////////////////////
-
-  override final def hasNext: Boolean = hasNewGroup
-
-  override final def next(): InternalRow = {
-    if (hasNext) {
-      // Process the current group.
-      processCurrentGroup()
-      // Generate output row for the current group.
-      val outputRow = generateOutput()
-      // Initilize buffer values for the next group.
-      initializeBuffer()
-
-      outputRow
-    } else {
-      // no more result
-      throw new NoSuchElementException
-    }
-  }
-
-  ///////////////////////////////////////////////////////////////////////////
-  // Methods that need to be implemented
-  ///////////////////////////////////////////////////////////////////////////
-
-  /** The initial input buffer offset for `inputBufferOffset` of an 
[[AggregateFunction2]]. */
-  protected def initialInputBufferOffset: Int
-
-  /** The function used to process an input row. */
-  protected def processRow(row: InternalRow): Unit
-
-  /** The function used to generate the result row. */
-  protected def generateOutput(): InternalRow
-
-  ///////////////////////////////////////////////////////////////////////////
-  // Initialize this iterator
-  ///////////////////////////////////////////////////////////////////////////
-
-  initialize()
-}
-
-/**
- * An iterator used to do partial aggregations (for those aggregate functions 
with mode Partial).
- * It assumes that input rows are already grouped by values of 
`groupingExpressions`.
- * The format of its output rows is:
- * |groupingExpr1|...|groupingExprN|aggregationBuffer1|...|aggregationBufferN|
- */
-class PartialSortAggregationIterator(
-    groupingExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[AggregateExpression2],
-    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
-    inputAttributes: Seq[Attribute],
-    inputIter: Iterator[InternalRow])
-  extends SortAggregationIterator(
-    groupingExpressions,
-    aggregateExpressions,
-    newMutableProjection,
-    inputAttributes,
-    inputIter) {
-
-  // This projection is used to update buffer values for all 
AlgebraicAggregates.
-  private val algebraicUpdateProjection = {
-    val bufferSchema = aggregateFunctions.flatMap(_.bufferAttributes)
-    val updateExpressions = aggregateFunctions.flatMap {
-      case ae: AlgebraicAggregate => ae.updateExpressions
-      case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
-    }
-    newMutableProjection(updateExpressions, bufferSchema ++ 
inputAttributes)().target(buffer)
-  }
-
-  override protected def initialInputBufferOffset: Int = 0
-
-  override protected def processRow(row: InternalRow): Unit = {
-    // Process all algebraic aggregate functions.
-    algebraicUpdateProjection(joinedRow(buffer, row))
-    // Process all non-algebraic aggregate functions.
-    var i = 0
-    while (i < nonAlgebraicAggregateFunctions.length) {
-      nonAlgebraicAggregateFunctions(i).update(buffer, row)
-      i += 1
-    }
-  }
-
-  override protected def generateOutput(): InternalRow = {
-    // We just output the grouping expressions and the underlying buffer.
-    joinedRow(currentGroupingKey, buffer).copy()
-  }
-}
-
-/**
- * An iterator used to do partial merge aggregations (for those aggregate 
functions with mode
- * PartialMerge). It assumes that input rows are already grouped by values of
- * `groupingExpressions`.
- * The format of its input rows is:
- * |groupingExpr1|...|groupingExprN|aggregationBuffer1|...|aggregationBufferN|
- *
- * The format of its internal buffer is:
- * |aggregationBuffer1|...|aggregationBufferN|
- *
- * The format of its output rows is:
- * |groupingExpr1|...|groupingExprN|aggregationBuffer1|...|aggregationBufferN|
- */
-class PartialMergeSortAggregationIterator(
-    groupingExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[AggregateExpression2],
-    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
-    inputAttributes: Seq[Attribute],
-    inputIter: Iterator[InternalRow])
-  extends SortAggregationIterator(
-    groupingExpressions,
-    aggregateExpressions,
-    newMutableProjection,
-    inputAttributes,
-    inputIter) {
-
-  // This projection is used to merge buffer values for all 
AlgebraicAggregates.
-  private val algebraicMergeProjection = {
-    val mergeInputSchema =
-      aggregateFunctions.flatMap(_.bufferAttributes) ++
-        groupingExpressions.map(_.toAttribute) ++
-        aggregateFunctions.flatMap(_.cloneBufferAttributes)
-    val mergeExpressions = aggregateFunctions.flatMap {
-      case ae: AlgebraicAggregate => ae.mergeExpressions
-      case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
-    }
-
-    newMutableProjection(mergeExpressions, mergeInputSchema)()
-  }
-
-  override protected def initialInputBufferOffset: Int = 
groupingExpressions.length
-
-  override protected def processRow(row: InternalRow): Unit = {
-    // Process all algebraic aggregate functions.
-    algebraicMergeProjection.target(buffer)(joinedRow(buffer, row))
-    // Process all non-algebraic aggregate functions.
-    var i = 0
-    while (i < nonAlgebraicAggregateFunctions.length) {
-      nonAlgebraicAggregateFunctions(i).merge(buffer, row)
-      i += 1
-    }
-  }
-
-  override protected def generateOutput(): InternalRow = {
-    // We output grouping expressions and aggregation buffers.
-    joinedRow(currentGroupingKey, buffer).copy()
-  }
-}
-
-/**
- * An iterator used to do final aggregations (for those aggregate functions 
with mode
- * Final). It assumes that input rows are already grouped by values of
- * `groupingExpressions`.
- * The format of its input rows is:
- * |groupingExpr1|...|groupingExprN|aggregationBuffer1|...|aggregationBufferN|
- *
- * The format of its internal buffer is:
- * |aggregationBuffer1|...|aggregationBufferN|
- *
- * The format of its output rows is represented by the schema of 
`resultExpressions`.
- */
-class FinalSortAggregationIterator(
-    groupingExpressions: Seq[NamedExpression],
-    aggregateExpressions: Seq[AggregateExpression2],
-    aggregateAttributes: Seq[Attribute],
-    resultExpressions: Seq[NamedExpression],
-    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
-    inputAttributes: Seq[Attribute],
-    inputIter: Iterator[InternalRow])
-  extends SortAggregationIterator(
-    groupingExpressions,
-    aggregateExpressions,
-    newMutableProjection,
-    inputAttributes,
-    inputIter) {
-
-  // The result of aggregate functions.
-  private val aggregateResult: MutableRow = new 
GenericMutableRow(aggregateAttributes.length)
-
-  // The projection used to generate the output rows of this operator.
-  // This is only used when we are generating final results of aggregate 
functions.
-  private val resultProjection =
-    newMutableProjection(
-      resultExpressions, groupingExpressions.map(_.toAttribute) ++ 
aggregateAttributes)()
-
-  // This projection is used to merge buffer values for all 
AlgebraicAggregates.
-  private val algebraicMergeProjection = {
-    val mergeInputSchema =
-      aggregateFunctions.flatMap(_.bufferAttributes) ++
-        groupingExpressions.map(_.toAttribute) ++
-        aggregateFunctions.flatMap(_.cloneBufferAttributes)
-    val mergeExpressions = aggregateFunctions.flatMap {
-      case ae: AlgebraicAggregate => ae.mergeExpressions
-      case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
-    }
-
-    newMutableProjection(mergeExpressions, mergeInputSchema)()
-  }
-
-  // This projection is used to evaluate all AlgebraicAggregates.
-  private val algebraicEvalProjection = {
-    val bufferSchemata = aggregateFunctions.flatMap(_.bufferAttributes)
-    val evalExpressions = aggregateFunctions.map {
-      case ae: AlgebraicAggregate => ae.evaluateExpression
-      case agg: AggregateFunction2 => NoOp
-    }
-
-    newMutableProjection(evalExpressions, bufferSchemata)()
-  }
-
-  override protected def initialInputBufferOffset: Int = 
groupingExpressions.length
-
-  override def initialize(): Unit = {
-    if (inputIter.hasNext) {
-      initializeBuffer()
-      val currentRow = inputIter.next().copy()
-      // partitionGenerator is a mutable projection. Since we need to track 
nextGroupingKey,
-      // we are making a copy at here.
-      nextGroupingKey = groupGenerator(currentRow).copy()
-      firstRowInNextGroup = currentRow
-    } else {
-      if (groupingExpressions.isEmpty) {
-        // If there is no grouping expression, we need to generate a single 
row as the output.
-        initializeBuffer()
-        // Right now, the buffer only contains initial buffer values. Because
-        // merging two buffers with initial values will generate a row that
-        // still store initial values. We set the currentRow as the copy of 
the current buffer.
-        // Because input aggregation buffer has initialInputBufferOffset extra 
values at the
-        // beginning, we create a dummy row for this part.
-        val currentRow =
-          joinedRow(new GenericInternalRow(initialInputBufferOffset), 
buffer).copy()
-        nextGroupingKey = groupGenerator(currentRow).copy()
-        firstRowInNextGroup = currentRow
-      } else {
-        // This iter is an empty one.
-        hasNewGroup = false
-      }
-    }
-  }
-
-  override protected def processRow(row: InternalRow): Unit = {
-    // Process all algebraic aggregate functions.
-    algebraicMergeProjection.target(buffer)(joinedRow(buffer, row))
-    // Process all non-algebraic aggregate functions.
-    var i = 0
-    while (i < nonAlgebraicAggregateFunctions.length) {
-      nonAlgebraicAggregateFunctions(i).merge(buffer, row)
-      i += 1
-    }
-  }
-
-  override protected def generateOutput(): InternalRow = {
-    // Generate results for all algebraic aggregate functions.
-    algebraicEvalProjection.target(aggregateResult)(buffer)
-    // Generate results for all non-algebraic aggregate functions.
-    var i = 0
-    while (i < nonAlgebraicAggregateFunctions.length) {
-      aggregateResult.update(
-        nonAlgebraicAggregateFunctionPositions(i),
-        nonAlgebraicAggregateFunctions(i).eval(buffer))
-      i += 1
-    }
-    resultProjection(joinedRow(currentGroupingKey, aggregateResult))
-  }
-}
-
-/**
- * An iterator used to do both final aggregations (for those aggregate 
functions with mode
- * Final) and complete aggregations (for those aggregate functions with mode 
Complete).
- * It assumes that input rows are already grouped by values of 
`groupingExpressions`.
- * The format of its input rows is:
- * 
|groupingExpr1|...|groupingExprN|col1|...|colM|aggregationBuffer1|...|aggregationBufferN|
- * col1 to colM are columns used by aggregate functions with Complete mode.
- * aggregationBuffer1 to aggregationBufferN are buffers used by aggregate 
functions with
- * Final mode.
- *
- * The format of its internal buffer is:
- * |aggregationBuffer1|...|aggregationBuffer(N+M)|
- * For aggregation buffers, first N aggregation buffers are used by N 
aggregate functions with
- * mode Final. Then, the last M aggregation buffers are used by M aggregate 
functions with mode
- * Complete.
- *
- * The format of its output rows is represented by the schema of 
`resultExpressions`.
- */
-class FinalAndCompleteSortAggregationIterator(
-    override protected val initialInputBufferOffset: Int,
-    groupingExpressions: Seq[NamedExpression],
-    finalAggregateExpressions: Seq[AggregateExpression2],
-    finalAggregateAttributes: Seq[Attribute],
-    completeAggregateExpressions: Seq[AggregateExpression2],
-    completeAggregateAttributes: Seq[Attribute],
-    resultExpressions: Seq[NamedExpression],
-    newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => 
MutableProjection),
-    inputAttributes: Seq[Attribute],
-    inputIter: Iterator[InternalRow])
-  extends SortAggregationIterator(
-    groupingExpressions,
-    // TODO: document the ordering
-    finalAggregateExpressions ++ completeAggregateExpressions,
-    newMutableProjection,
-    inputAttributes,
-    inputIter) {
-
-  // The result of aggregate functions.
-  private val aggregateResult: MutableRow =
-    new GenericMutableRow(completeAggregateAttributes.length + 
finalAggregateAttributes.length)
-
-  // The projection used to generate the output rows of this operator.
-  // This is only used when we are generating final results of aggregate 
functions.
-  private val resultProjection = {
-    val inputSchema =
-      groupingExpressions.map(_.toAttribute) ++
-        finalAggregateAttributes ++
-        completeAggregateAttributes
-    newMutableProjection(resultExpressions, inputSchema)()
-  }
-
-  // All aggregate functions with mode Final.
-  private val finalAggregateFunctions: Array[AggregateFunction2] = {
-    val functions = new 
Array[AggregateFunction2](finalAggregateExpressions.length)
-    var i = 0
-    while (i < finalAggregateExpressions.length) {
-      functions(i) = aggregateFunctions(i)
-      i += 1
-    }
-    functions
-  }
-
-  // All non-algebraic aggregate functions with mode Final.
-  private val finalNonAlgebraicAggregateFunctions: Array[AggregateFunction2] =
-    finalAggregateFunctions.collect {
-      case func: AggregateFunction2 if !func.isInstanceOf[AlgebraicAggregate] 
=> func
-    }
-
-  // All aggregate functions with mode Complete.
-  private val completeAggregateFunctions: Array[AggregateFunction2] = {
-    val functions = new 
Array[AggregateFunction2](completeAggregateExpressions.length)
-    var i = 0
-    while (i < completeAggregateExpressions.length) {
-      functions(i) = aggregateFunctions(finalAggregateFunctions.length + i)
-      i += 1
-    }
-    functions
-  }
-
-  // All non-algebraic aggregate functions with mode Complete.
-  private val completeNonAlgebraicAggregateFunctions: 
Array[AggregateFunction2] =
-    completeAggregateFunctions.collect {
-      case func: AggregateFunction2 if !func.isInstanceOf[AlgebraicAggregate] 
=> func
-    }
-
-  // This projection is used to merge buffer values for all 
AlgebraicAggregates with mode
-  // Final.
-  private val finalAlgebraicMergeProjection = {
-    // The first initialInputBufferOffset values of the input aggregation 
buffer is
-    // for grouping expressions and distinct columns.
-    val groupingAttributesAndDistinctColumns = 
inputAttributes.take(initialInputBufferOffset)
-
-    val completeOffsetExpressions =
-      
Seq.fill(completeAggregateFunctions.map(_.bufferAttributes.length).sum)(NoOp)
-
-    val mergeInputSchema =
-      finalAggregateFunctions.flatMap(_.bufferAttributes) ++
-        completeAggregateFunctions.flatMap(_.bufferAttributes) ++
-        groupingAttributesAndDistinctColumns ++
-        finalAggregateFunctions.flatMap(_.cloneBufferAttributes)
-    val mergeExpressions =
-      finalAggregateFunctions.flatMap {
-        case ae: AlgebraicAggregate => ae.mergeExpressions
-        case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
-      } ++ completeOffsetExpressions
-    newMutableProjection(mergeExpressions, mergeInputSchema)()
-  }
-
-  // This projection is used to update buffer values for all 
AlgebraicAggregates with mode
-  // Complete.
-  private val completeAlgebraicUpdateProjection = {
-    // We do not touch buffer values of aggregate functions with the Final 
mode.
-    val finalOffsetExpressions =
-      
Seq.fill(finalAggregateFunctions.map(_.bufferAttributes.length).sum)(NoOp)
-
-    val bufferSchema =
-      finalAggregateFunctions.flatMap(_.bufferAttributes) ++
-        completeAggregateFunctions.flatMap(_.bufferAttributes)
-    val updateExpressions =
-      finalOffsetExpressions ++ completeAggregateFunctions.flatMap {
-        case ae: AlgebraicAggregate => ae.updateExpressions
-        case agg: AggregateFunction2 => 
Seq.fill(agg.bufferAttributes.length)(NoOp)
-      }
-    newMutableProjection(updateExpressions, bufferSchema ++ 
inputAttributes)().target(buffer)
-  }
-
-  // This projection is used to evaluate all AlgebraicAggregates.
-  private val algebraicEvalProjection = {
-    val bufferSchemata = aggregateFunctions.flatMap(_.bufferAttributes)
-    val evalExpressions = aggregateFunctions.map {
-      case ae: AlgebraicAggregate => ae.evaluateExpression
-      case agg: AggregateFunction2 => NoOp
-    }
-
-    newMutableProjection(evalExpressions, bufferSchemata)()
-  }
-
-  override def initialize(): Unit = {
-    if (inputIter.hasNext) {
-      initializeBuffer()
-      val currentRow = inputIter.next().copy()
-      // partitionGenerator is a mutable projection. Since we need to track 
nextGroupingKey,
-      // we are making a copy at here.
-      nextGroupingKey = groupGenerator(currentRow).copy()
-      firstRowInNextGroup = currentRow
-    } else {
-      if (groupingExpressions.isEmpty) {
-        // If there is no grouping expression, we need to generate a single 
row as the output.
-        initializeBuffer()
-        // Right now, the buffer only contains initial buffer values. Because
-        // merging two buffers with initial values will generate a row that
-        // still store initial values. We set the currentRow as the copy of 
the current buffer.
-        // Because input aggregation buffer has initialInputBufferOffset extra 
values at the
-        // beginning, we create a dummy row for this part.
-        val currentRow =
-          joinedRow(new GenericInternalRow(initialInputBufferOffset), 
buffer).copy()
-        nextGroupingKey = groupGenerator(currentRow).copy()
-        firstRowInNextGroup = currentRow
-      } else {
-        // This iter is an empty one.
-        hasNewGroup = false
-      }
-    }
-  }
-
-  override protected def processRow(row: InternalRow): Unit = {
-    val input = joinedRow(buffer, row)
-    // For all aggregate functions with mode Complete, update buffers.
-    completeAlgebraicUpdateProjection(input)
-    var i = 0
-    while (i < completeNonAlgebraicAggregateFunctions.length) {
-      completeNonAlgebraicAggregateFunctions(i).update(buffer, row)
-      i += 1
-    }
-
-    // For all aggregate functions with mode Final, merge buffers.
-    finalAlgebraicMergeProjection.target(buffer)(input)
-    i = 0
-    while (i < finalNonAlgebraicAggregateFunctions.length) {
-      finalNonAlgebraicAggregateFunctions(i).merge(buffer, row)
-      i += 1
-    }
-  }
-
-  override protected def generateOutput(): InternalRow = {
-    // Generate results for all algebraic aggregate functions.
-    algebraicEvalProjection.target(aggregateResult)(buffer)
-    // Generate results for all non-algebraic aggregate functions.
-    var i = 0
-    while (i < nonAlgebraicAggregateFunctions.length) {
-      aggregateResult.update(
-        nonAlgebraicAggregateFunctionPositions(i),
-        nonAlgebraicAggregateFunctions(i).eval(buffer))
-      i += 1
-    }
-
-    resultProjection(joinedRow(currentGroupingKey, aggregateResult))
-  }
-}

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
index cc54319..5fafc91 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/udaf.scala
@@ -24,7 +24,154 @@ import 
org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjecti
 import org.apache.spark.sql.catalyst.expressions.{MutableRow, 
InterpretedMutableProjection, AttributeReference, Expression}
 import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateFunction2
 import org.apache.spark.sql.expressions.{MutableAggregationBuffer, 
UserDefinedAggregateFunction}
-import org.apache.spark.sql.types.{Metadata, StructField, StructType, DataType}
+import org.apache.spark.sql.types._
+
+/**
+ * A helper trait used to create specialized setter and getter for types 
supported by
+ * [[org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap]]'s buffer.
+ * (see UnsafeFixedWidthAggregationMap.supportsAggregationBufferSchema).
+ */
+sealed trait BufferSetterGetterUtils {
+
+  def createGetters(schema: StructType): Array[(InternalRow, Int) => Any] = {
+    val dataTypes = schema.fields.map(_.dataType)
+    val getters = new Array[(InternalRow, Int) => Any](dataTypes.length)
+
+    var i = 0
+    while (i < getters.length) {
+      getters(i) = dataTypes(i) match {
+        case BooleanType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getBoolean(ordinal)
+
+        case ByteType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getByte(ordinal)
+
+        case ShortType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getShort(ordinal)
+
+        case IntegerType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getInt(ordinal)
+
+        case LongType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getLong(ordinal)
+
+        case FloatType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getFloat(ordinal)
+
+        case DoubleType =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getDouble(ordinal)
+
+        case dt: DecimalType =>
+          val precision = dt.precision
+          val scale = dt.scale
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.getDecimal(ordinal, 
precision, scale)
+
+        case other =>
+          (row: InternalRow, ordinal: Int) =>
+            if (row.isNullAt(ordinal)) null else row.get(ordinal, other)
+      }
+
+      i += 1
+    }
+
+    getters
+  }
+
+  def createSetters(schema: StructType): Array[((MutableRow, Int, Any) => 
Unit)] = {
+    val dataTypes = schema.fields.map(_.dataType)
+    val setters = new Array[(MutableRow, Int, Any) => Unit](dataTypes.length)
+
+    var i = 0
+    while (i < setters.length) {
+      setters(i) = dataTypes(i) match {
+        case b: BooleanType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setBoolean(ordinal, value.asInstanceOf[Boolean])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case ByteType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setByte(ordinal, value.asInstanceOf[Byte])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case ShortType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setShort(ordinal, value.asInstanceOf[Short])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case IntegerType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setInt(ordinal, value.asInstanceOf[Int])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case LongType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setLong(ordinal, value.asInstanceOf[Long])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case FloatType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setFloat(ordinal, value.asInstanceOf[Float])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case DoubleType =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setDouble(ordinal, value.asInstanceOf[Double])
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case dt: DecimalType =>
+          val precision = dt.precision
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.setDecimal(ordinal, value.asInstanceOf[Decimal], precision)
+            } else {
+              row.setNullAt(ordinal)
+            }
+
+        case other =>
+          (row: MutableRow, ordinal: Int, value: Any) =>
+            if (value != null) {
+              row.update(ordinal, value)
+            } else {
+              row.setNullAt(ordinal)
+            }
+      }
+
+      i += 1
+    }
+
+    setters
+  }
+}
 
 /**
  * A Mutable [[Row]] representing an mutable aggregation buffer.
@@ -35,7 +182,7 @@ private[sql] class MutableAggregationBufferImpl (
     toScalaConverters: Array[Any => Any],
     bufferOffset: Int,
     var underlyingBuffer: MutableRow)
-  extends MutableAggregationBuffer {
+  extends MutableAggregationBuffer with BufferSetterGetterUtils {
 
   private[this] val offsets: Array[Int] = {
     val newOffsets = new Array[Int](length)
@@ -47,6 +194,10 @@ private[sql] class MutableAggregationBufferImpl (
     newOffsets
   }
 
+  private[this] val bufferValueGetters = createGetters(schema)
+
+  private[this] val bufferValueSetters = createSetters(schema)
+
   override def length: Int = toCatalystConverters.length
 
   override def get(i: Int): Any = {
@@ -54,7 +205,7 @@ private[sql] class MutableAggregationBufferImpl (
       throw new IllegalArgumentException(
         s"Could not access ${i}th value in this buffer because it only has 
$length values.")
     }
-    toScalaConverters(i)(underlyingBuffer.get(offsets(i), schema(i).dataType))
+    toScalaConverters(i)(bufferValueGetters(i)(underlyingBuffer, offsets(i)))
   }
 
   def update(i: Int, value: Any): Unit = {
@@ -62,7 +213,15 @@ private[sql] class MutableAggregationBufferImpl (
       throw new IllegalArgumentException(
         s"Could not update ${i}th value in this buffer because it only has 
$length values.")
     }
-    underlyingBuffer.update(offsets(i), toCatalystConverters(i)(value))
+
+    bufferValueSetters(i)(underlyingBuffer, offsets(i), 
toCatalystConverters(i)(value))
+  }
+
+  // Because get method call specialized getter based on the schema, we cannot 
use the
+  // default implementation of the isNullAt (which is get(i) == null).
+  // We have to override it to call isNullAt of the underlyingBuffer.
+  override def isNullAt(i: Int): Boolean = {
+    underlyingBuffer.isNullAt(offsets(i))
   }
 
   override def copy(): MutableAggregationBufferImpl = {
@@ -84,7 +243,7 @@ private[sql] class InputAggregationBuffer private[sql] (
     toScalaConverters: Array[Any => Any],
     bufferOffset: Int,
     var underlyingInputBuffer: InternalRow)
-  extends Row {
+  extends Row with BufferSetterGetterUtils {
 
   private[this] val offsets: Array[Int] = {
     val newOffsets = new Array[Int](length)
@@ -96,6 +255,10 @@ private[sql] class InputAggregationBuffer private[sql] (
     newOffsets
   }
 
+  private[this] val bufferValueGetters = createGetters(schema)
+
+  def getBufferOffset: Int = bufferOffset
+
   override def length: Int = toCatalystConverters.length
 
   override def get(i: Int): Any = {
@@ -103,8 +266,14 @@ private[sql] class InputAggregationBuffer private[sql] (
       throw new IllegalArgumentException(
         s"Could not access ${i}th value in this buffer because it only has 
$length values.")
     }
-    // TODO: Use buffer schema to avoid using generic getter.
-    toScalaConverters(i)(underlyingInputBuffer.get(offsets(i), 
schema(i).dataType))
+    toScalaConverters(i)(bufferValueGetters(i)(underlyingInputBuffer, 
offsets(i)))
+  }
+
+  // Because get method call specialized getter based on the schema, we cannot 
use the
+  // default implementation of the isNullAt (which is get(i) == null).
+  // We have to override it to call isNullAt of the underlyingInputBuffer.
+  override def isNullAt(i: Int): Boolean = {
+    underlyingInputBuffer.isNullAt(offsets(i))
   }
 
   override def copy(): InputAggregationBuffer = {
@@ -147,7 +316,7 @@ private[sql] case class ScalaUDAF(
 
   override lazy val cloneBufferAttributes = 
bufferAttributes.map(_.newInstance())
 
-  val childrenSchema: StructType = {
+  private[this] val childrenSchema: StructType = {
     val inputFields = children.zipWithIndex.map {
       case (child, index) =>
         StructField(s"input$index", child.dataType, child.nullable, 
Metadata.empty)
@@ -155,7 +324,7 @@ private[sql] case class ScalaUDAF(
     StructType(inputFields)
   }
 
-  lazy val inputProjection = {
+  private lazy val inputProjection = {
     val inputAttributes = childrenSchema.toAttributes
     log.debug(
       s"Creating MutableProj: $children, inputSchema: $inputAttributes.")
@@ -168,40 +337,68 @@ private[sql] case class ScalaUDAF(
     }
   }
 
-  val inputToScalaConverters: Any => Any =
+  private[this] val inputToScalaConverters: Any => Any =
     CatalystTypeConverters.createToScalaConverter(childrenSchema)
 
-  val bufferValuesToCatalystConverters: Array[Any => Any] = 
bufferSchema.fields.map { field =>
-    CatalystTypeConverters.createToCatalystConverter(field.dataType)
+  private[this] val bufferValuesToCatalystConverters: Array[Any => Any] = {
+    bufferSchema.fields.map { field =>
+      CatalystTypeConverters.createToCatalystConverter(field.dataType)
+    }
   }
 
-  val bufferValuesToScalaConverters: Array[Any => Any] = 
bufferSchema.fields.map { field =>
-    CatalystTypeConverters.createToScalaConverter(field.dataType)
+  private[this] val bufferValuesToScalaConverters: Array[Any => Any] = {
+    bufferSchema.fields.map { field =>
+      CatalystTypeConverters.createToScalaConverter(field.dataType)
+    }
   }
 
-  lazy val inputAggregateBuffer: InputAggregationBuffer =
-    new InputAggregationBuffer(
-      bufferSchema,
-      bufferValuesToCatalystConverters,
-      bufferValuesToScalaConverters,
-      inputBufferOffset,
-      null)
-
-  lazy val mutableAggregateBuffer: MutableAggregationBufferImpl =
-    new MutableAggregationBufferImpl(
-      bufferSchema,
-      bufferValuesToCatalystConverters,
-      bufferValuesToScalaConverters,
-      mutableBufferOffset,
-      null)
+  // This buffer is only used at executor side.
+  private[this] var inputAggregateBuffer: InputAggregationBuffer = null
+
+  // This buffer is only used at executor side.
+  private[this] var mutableAggregateBuffer: MutableAggregationBufferImpl = null
+
+  // This buffer is only used at executor side.
+  private[this] var evalAggregateBuffer: InputAggregationBuffer = null
+
+  /**
+   * Sets the inputBufferOffset to newInputBufferOffset and then create a new 
instance of
+   * `inputAggregateBuffer` based on this new inputBufferOffset.
+   */
+  override def withNewInputBufferOffset(newInputBufferOffset: Int): Unit = {
+    super.withNewInputBufferOffset(newInputBufferOffset)
+    // inputBufferOffset has been updated.
+    inputAggregateBuffer =
+      new InputAggregationBuffer(
+        bufferSchema,
+        bufferValuesToCatalystConverters,
+        bufferValuesToScalaConverters,
+        inputBufferOffset,
+        null)
+  }
 
-  lazy val evalAggregateBuffer: InputAggregationBuffer =
-    new InputAggregationBuffer(
-      bufferSchema,
-      bufferValuesToCatalystConverters,
-      bufferValuesToScalaConverters,
-      mutableBufferOffset,
-      null)
+  /**
+   * Sets the mutableBufferOffset to newMutableBufferOffset and then create a 
new instance of
+   * `mutableAggregateBuffer` and `evalAggregateBuffer` based on this new 
mutableBufferOffset.
+   */
+  override def withNewMutableBufferOffset(newMutableBufferOffset: Int): Unit = 
{
+    super.withNewMutableBufferOffset(newMutableBufferOffset)
+    // mutableBufferOffset has been updated.
+    mutableAggregateBuffer =
+      new MutableAggregationBufferImpl(
+        bufferSchema,
+        bufferValuesToCatalystConverters,
+        bufferValuesToScalaConverters,
+        mutableBufferOffset,
+        null)
+    evalAggregateBuffer =
+      new InputAggregationBuffer(
+        bufferSchema,
+        bufferValuesToCatalystConverters,
+        bufferValuesToScalaConverters,
+        mutableBufferOffset,
+        null)
+  }
 
   override def initialize(buffer: MutableRow): Unit = {
     mutableAggregateBuffer.underlyingBuffer = buffer

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala
index 03635ba..960be08 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/utils.scala
@@ -17,13 +17,9 @@
 
 package org.apache.spark.sql.execution.aggregate
 
-import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan}
 import org.apache.spark.sql.execution.SparkPlan
-import org.apache.spark.sql.types.{StructType, MapType, ArrayType}
 
 /**
  * Utility functions used by the query planner to convert our plan to new 
aggregation code path.
@@ -52,13 +48,16 @@ object Utils {
       agg.aggregateFunction.bufferAttributes
     }
     val partialAggregate =
-      Aggregate2Sort(
-        None: Option[Seq[Expression]],
-        namedGroupingExpressions.map(_._2),
-        partialAggregateExpressions,
-        partialAggregateAttributes,
-        namedGroupingAttributes ++ partialAggregateAttributes,
-        child)
+      Aggregate(
+        requiredChildDistributionExpressions = None: Option[Seq[Expression]],
+        groupingExpressions = namedGroupingExpressions.map(_._2),
+        nonCompleteAggregateExpressions = partialAggregateExpressions,
+        nonCompleteAggregateAttributes = partialAggregateAttributes,
+        completeAggregateExpressions = Nil,
+        completeAggregateAttributes = Nil,
+        initialInputBufferOffset = 0,
+        resultExpressions = namedGroupingAttributes ++ 
partialAggregateAttributes,
+        child = child)
 
     // 2. Create an Aggregate Operator for final aggregations.
     val finalAggregateExpressions = aggregateExpressions.map(_.copy(mode = 
Final))
@@ -78,13 +77,17 @@ object Utils {
           }.getOrElse(expression)
       }.asInstanceOf[NamedExpression]
     }
-    val finalAggregate = Aggregate2Sort(
-      Some(namedGroupingAttributes),
-      namedGroupingAttributes,
-      finalAggregateExpressions,
-      finalAggregateAttributes,
-      rewrittenResultExpressions,
-      partialAggregate)
+    val finalAggregate =
+      Aggregate(
+        requiredChildDistributionExpressions = Some(namedGroupingAttributes),
+        groupingExpressions = namedGroupingAttributes,
+        nonCompleteAggregateExpressions = finalAggregateExpressions,
+        nonCompleteAggregateAttributes = finalAggregateAttributes,
+        completeAggregateExpressions = Nil,
+        completeAggregateAttributes = Nil,
+        initialInputBufferOffset = namedGroupingAttributes.length,
+        resultExpressions = rewrittenResultExpressions,
+        child = partialAggregate)
 
     finalAggregate :: Nil
   }
@@ -133,14 +136,21 @@ object Utils {
     val partialAggregateAttributes = partialAggregateExpressions.flatMap { agg 
=>
       agg.aggregateFunction.bufferAttributes
     }
+    val partialAggregateGroupingExpressions =
+      (namedGroupingExpressions ++ namedDistinctColumnExpressions).map(_._2)
+    val partialAggregateResult =
+      namedGroupingAttributes ++ distinctColumnAttributes ++ 
partialAggregateAttributes
     val partialAggregate =
-      Aggregate2Sort(
-        None: Option[Seq[Expression]],
-        (namedGroupingExpressions ++ namedDistinctColumnExpressions).map(_._2),
-        partialAggregateExpressions,
-        partialAggregateAttributes,
-        namedGroupingAttributes ++ distinctColumnAttributes ++ 
partialAggregateAttributes,
-        child)
+      Aggregate(
+        requiredChildDistributionExpressions = None: Option[Seq[Expression]],
+        groupingExpressions = partialAggregateGroupingExpressions,
+        nonCompleteAggregateExpressions = partialAggregateExpressions,
+        nonCompleteAggregateAttributes = partialAggregateAttributes,
+        completeAggregateExpressions = Nil,
+        completeAggregateAttributes = Nil,
+        initialInputBufferOffset = 0,
+        resultExpressions = partialAggregateResult,
+        child = child)
 
     // 2. Create an Aggregate Operator for partial merge aggregations.
     val partialMergeAggregateExpressions = functionsWithoutDistinct.map {
@@ -151,14 +161,19 @@ object Utils {
       partialMergeAggregateExpressions.flatMap { agg =>
         agg.aggregateFunction.bufferAttributes
       }
+    val partialMergeAggregateResult =
+      namedGroupingAttributes ++ distinctColumnAttributes ++ 
partialMergeAggregateAttributes
     val partialMergeAggregate =
-      Aggregate2Sort(
-        Some(namedGroupingAttributes),
-        namedGroupingAttributes ++ distinctColumnAttributes,
-        partialMergeAggregateExpressions,
-        partialMergeAggregateAttributes,
-        namedGroupingAttributes ++ distinctColumnAttributes ++ 
partialMergeAggregateAttributes,
-        partialAggregate)
+      Aggregate(
+        requiredChildDistributionExpressions = Some(namedGroupingAttributes),
+        groupingExpressions = namedGroupingAttributes ++ 
distinctColumnAttributes,
+        nonCompleteAggregateExpressions = partialMergeAggregateExpressions,
+        nonCompleteAggregateAttributes = partialMergeAggregateAttributes,
+        completeAggregateExpressions = Nil,
+        completeAggregateAttributes = Nil,
+        initialInputBufferOffset = (namedGroupingAttributes ++ 
distinctColumnAttributes).length,
+        resultExpressions = partialMergeAggregateResult,
+        child = partialAggregate)
 
     // 3. Create an Aggregate Operator for partial merge aggregations.
     val finalAggregateExpressions = functionsWithoutDistinct.map {
@@ -199,15 +214,17 @@ object Utils {
           }.getOrElse(expression)
       }.asInstanceOf[NamedExpression]
     }
-    val finalAndCompleteAggregate = FinalAndCompleteAggregate2Sort(
-      namedGroupingAttributes ++ distinctColumnAttributes,
-      namedGroupingAttributes,
-      finalAggregateExpressions,
-      finalAggregateAttributes,
-      completeAggregateExpressions,
-      completeAggregateAttributes,
-      rewrittenResultExpressions,
-      partialMergeAggregate)
+    val finalAndCompleteAggregate =
+      Aggregate(
+        requiredChildDistributionExpressions = Some(namedGroupingAttributes),
+        groupingExpressions = namedGroupingAttributes,
+        nonCompleteAggregateExpressions = finalAggregateExpressions,
+        nonCompleteAggregateAttributes = finalAggregateAttributes,
+        completeAggregateExpressions = completeAggregateExpressions,
+        completeAggregateAttributes = completeAggregateAttributes,
+        initialInputBufferOffset = (namedGroupingAttributes ++ 
distinctColumnAttributes).length,
+        resultExpressions = rewrittenResultExpressions,
+        child = partialMergeAggregate)
 
     finalAndCompleteAggregate :: Nil
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index 2294a67..5a1b000 100644
--- 
a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ 
b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -220,7 +220,6 @@ case class TakeOrderedAndProject(
   override def outputOrdering: Seq[SortOrder] = sortOrder
 }
 
-
 /**
  * :: DeveloperApi ::
  * Return a new RDD that has exactly `numPartitions` partitions.

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
index 51fe9d9..bbadc20 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
@@ -17,14 +17,14 @@
 
 package org.apache.spark.sql
 
-import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
-import org.scalatest.BeforeAndAfterAll
-
 import java.sql.Timestamp
 
+import org.scalatest.BeforeAndAfterAll
+
+import org.apache.spark.sql.catalyst.analysis.FunctionRegistry
 import org.apache.spark.sql.catalyst.DefaultParserDialect
 import org.apache.spark.sql.catalyst.errors.DialectException
-import org.apache.spark.sql.execution.aggregate.Aggregate2Sort
+import org.apache.spark.sql.execution.aggregate
 import org.apache.spark.sql.execution.GeneratedAggregate
 import org.apache.spark.sql.functions._
 import org.apache.spark.sql.TestData._
@@ -273,7 +273,7 @@ class SQLQuerySuite extends QueryTest with 
BeforeAndAfterAll with SQLTestUtils {
       var hasGeneratedAgg = false
       df.queryExecution.executedPlan.foreach {
         case generatedAgg: GeneratedAggregate => hasGeneratedAgg = true
-        case newAggregate: Aggregate2Sort => hasGeneratedAgg = true
+        case newAggregate: aggregate.Aggregate => hasGeneratedAgg = true
         case _ =>
       }
       if (!hasGeneratedAgg) {

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala
----------------------------------------------------------------------
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala
index 54f82f8..7978ed5 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlSerializer2Suite.scala
@@ -138,7 +138,14 @@ abstract class SparkSqlSerializer2Suite extends QueryTest 
with BeforeAndAfterAll
           s"Expected $expectedSerializerClass as the serializer of Exchange. " 
+
           s"However, the serializer was not set."
         val serializer = 
dependency.serializer.getOrElse(fail(serializerNotSetMessage))
-        assert(serializer.getClass === expectedSerializerClass)
+        val isExpectedSerializer =
+          serializer.getClass == expectedSerializerClass ||
+            serializer.getClass == classOf[UnsafeRowSerializer]
+        val wrongSerializerErrorMessage =
+          s"Expected ${expectedSerializerClass.getCanonicalName} or " +
+            s"${classOf[UnsafeRowSerializer].getCanonicalName}. But " +
+            s"${serializer.getClass.getCanonicalName} is used."
+        assert(isExpectedSerializer, wrongSerializerErrorMessage)
       case _ => // Ignore other nodes.
     }
   }

http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
index 0375eb7..6f0db27 100644
--- 
a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
+++ 
b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala
@@ -17,15 +17,15 @@
 
 package org.apache.spark.sql.hive.execution
 
-import org.apache.spark.sql.execution.aggregate.Aggregate2Sort
+import org.apache.spark.sql.execution.aggregate
 import org.apache.spark.sql.hive.test.TestHive
 import org.apache.spark.sql.test.SQLTestUtils
 import org.apache.spark.sql.types.{IntegerType, StringType, StructField, 
StructType}
-import org.apache.spark.sql.{AnalysisException, QueryTest, Row}
+import org.apache.spark.sql.{SQLConf, AnalysisException, QueryTest, Row}
 import org.scalatest.BeforeAndAfterAll
 import test.org.apache.spark.sql.hive.aggregate.{MyDoubleAvg, MyDoubleSum}
 
-class AggregationQuerySuite extends QueryTest with SQLTestUtils with 
BeforeAndAfterAll {
+abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with 
BeforeAndAfterAll {
 
   override val sqlContext = TestHive
   import sqlContext.implicits._
@@ -34,7 +34,7 @@ class AggregationQuerySuite extends QueryTest with 
SQLTestUtils with BeforeAndAf
 
   override def beforeAll(): Unit = {
     originalUseAggregate2 = sqlContext.conf.useSqlAggregate2
-    sqlContext.sql("set spark.sql.useAggregate2=true")
+    sqlContext.setConf(SQLConf.USE_SQL_AGGREGATE2.key, "true")
     val data1 = Seq[(Integer, Integer)](
       (1, 10),
       (null, -60),
@@ -81,7 +81,7 @@ class AggregationQuerySuite extends QueryTest with 
SQLTestUtils with BeforeAndAf
     sqlContext.sql("DROP TABLE IF EXISTS agg1")
     sqlContext.sql("DROP TABLE IF EXISTS agg2")
     sqlContext.dropTempTable("emptyTable")
-    sqlContext.sql(s"set spark.sql.useAggregate2=$originalUseAggregate2")
+    sqlContext.setConf(SQLConf.USE_SQL_AGGREGATE2.key, 
originalUseAggregate2.toString)
   }
 
   test("empty table") {
@@ -454,54 +454,86 @@ class AggregationQuerySuite extends QueryTest with 
SQLTestUtils with BeforeAndAf
   }
 
   test("error handling") {
-    sqlContext.sql(s"set spark.sql.useAggregate2=false")
-    var errorMessage = intercept[AnalysisException] {
-      sqlContext.sql(
-        """
-          |SELECT
-          |  key,
-          |  sum(value + 1.5 * key),
-          |  mydoublesum(value),
-          |  mydoubleavg(value)
-          |FROM agg1
-          |GROUP BY key
-        """.stripMargin).collect()
-    }.getMessage
-    assert(errorMessage.contains("implemented based on the new Aggregate 
Function interface"))
+    withSQLConf("spark.sql.useAggregate2" -> "false") {
+      val errorMessage = intercept[AnalysisException] {
+        sqlContext.sql(
+          """
+            |SELECT
+            |  key,
+            |  sum(value + 1.5 * key),
+            |  mydoublesum(value),
+            |  mydoubleavg(value)
+            |FROM agg1
+            |GROUP BY key
+          """.stripMargin).collect()
+      }.getMessage
+      assert(errorMessage.contains("implemented based on the new Aggregate 
Function interface"))
+    }
 
     // TODO: once we support Hive UDAF in the new interface,
     // we can remove the following two tests.
-    sqlContext.sql(s"set spark.sql.useAggregate2=true")
-    errorMessage = intercept[AnalysisException] {
-      sqlContext.sql(
+    withSQLConf("spark.sql.useAggregate2" -> "true") {
+      val errorMessage = intercept[AnalysisException] {
+        sqlContext.sql(
+          """
+            |SELECT
+            |  key,
+            |  mydoublesum(value + 1.5 * key),
+            |  stddev_samp(value)
+            |FROM agg1
+            |GROUP BY key
+          """.stripMargin).collect()
+      }.getMessage
+      assert(errorMessage.contains("implemented based on the new Aggregate 
Function interface"))
+
+      // This will fall back to the old aggregate
+      val newAggregateOperators = sqlContext.sql(
         """
           |SELECT
           |  key,
-          |  mydoublesum(value + 1.5 * key),
+          |  sum(value + 1.5 * key),
           |  stddev_samp(value)
           |FROM agg1
           |GROUP BY key
-        """.stripMargin).collect()
-    }.getMessage
-    assert(errorMessage.contains("implemented based on the new Aggregate 
Function interface"))
-
-    // This will fall back to the old aggregate
-    val newAggregateOperators = sqlContext.sql(
-      """
-        |SELECT
-        |  key,
-        |  sum(value + 1.5 * key),
-        |  stddev_samp(value)
-        |FROM agg1
-        |GROUP BY key
-      """.stripMargin).queryExecution.executedPlan.collect {
-      case agg: Aggregate2Sort => agg
+        """.stripMargin).queryExecution.executedPlan.collect {
+        case agg: aggregate.Aggregate => agg
+      }
+      val message =
+        "We should fallback to the old aggregation code path if " +
+          "there is any aggregate function that cannot be converted to the new 
interface."
+      assert(newAggregateOperators.isEmpty, message)
     }
-    val message =
-      "We should fallback to the old aggregation code path if there is any 
aggregate function " +
-        "that cannot be converted to the new interface."
-    assert(newAggregateOperators.isEmpty, message)
+  }
+}
+
+class SortBasedAggregationQuerySuite extends AggregationQuerySuite {
 
-    sqlContext.sql(s"set spark.sql.useAggregate2=true")
+  var originalUnsafeEnabled: Boolean = _
+
+  override def beforeAll(): Unit = {
+    originalUnsafeEnabled = sqlContext.conf.unsafeEnabled
+    sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, "false")
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, 
originalUnsafeEnabled.toString)
+  }
+}
+
+class TungstenAggregationQuerySuite extends AggregationQuerySuite {
+
+  var originalUnsafeEnabled: Boolean = _
+
+  override def beforeAll(): Unit = {
+    originalUnsafeEnabled = sqlContext.conf.unsafeEnabled
+    sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, "true")
+    super.beforeAll()
+  }
+
+  override def afterAll(): Unit = {
+    super.afterAll()
+    sqlContext.setConf(SQLConf.UNSAFE_ENABLED.key, 
originalUnsafeEnabled.toString)
   }
 }


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

Reply via email to