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