[SPARK-9240] [SQL] Hybrid aggregate operator using unsafe row This PR adds a base aggregation iterator `AggregationIterator`, which is used to create `SortBasedAggregationIterator` (for sort-based aggregation) and `UnsafeHybridAggregationIterator` (first it tries hash-based aggregation and falls back to the sort-based aggregation (using external sorter) if we cannot allocate memory for the map). With these two iterators, we will not need existing iterators and I am removing those. Also, we can use a single physical `Aggregate` operator and it internally determines what iterators to used.
https://issues.apache.org/jira/browse/SPARK-9240 Author: Yin Huai <yh...@databricks.com> Closes #7813 from yhuai/AggregateOperator and squashes the following commits: e317e2b [Yin Huai] Remove unnecessary change. 74d93c5 [Yin Huai] Merge remote-tracking branch 'upstream/master' into AggregateOperator ba6afbc [Yin Huai] Add a little bit more comments. c9cf3b6 [Yin Huai] update 0f1b06f [Yin Huai] Remove unnecessary code. 21fd15f [Yin Huai] Remove unnecessary change. 964f88b [Yin Huai] Implement fallback strategy. b1ea5cf [Yin Huai] wip 7fcbd87 [Yin Huai] Add a flag to control what iterator to use. 533d5b2 [Yin Huai] Prepare for fallback! 33b7022 [Yin Huai] wip bd9282b [Yin Huai] UDAFs now supports UnsafeRow. f52ee53 [Yin Huai] wip 3171f44 [Yin Huai] wip d2c45a0 [Yin Huai] wip f60cc83 [Yin Huai] Also check input schema. af32210 [Yin Huai] Check iter.hasNext before we create an iterator because the constructor of the iterato will read at least one row from a non-empty input iter. 299008c [Yin Huai] First round cleanup. 3915bac [Yin Huai] Create a base iterator class for aggregation iterators and add the initial version of the hybrid iterator. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1ebd41b1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1ebd41b1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1ebd41b1 Branch: refs/heads/master Commit: 1ebd41b141a95ec264bd2dd50f0fe24cd459035d Parents: 98d6d9c Author: Yin Huai <yh...@databricks.com> Authored: Mon Aug 3 00:23:08 2015 -0700 Committer: Reynold Xin <r...@databricks.com> Committed: Mon Aug 3 00:23:08 2015 -0700 ---------------------------------------------------------------------- .../expressions/aggregate/interfaces.scala | 19 +- .../sql/execution/aggregate/Aggregate.scala | 182 +++++ .../aggregate/AggregationIterator.scala | 490 ++++++++++++++ .../SortBasedAggregationIterator.scala | 236 +++++++ .../UnsafeHybridAggregationIterator.scala | 398 +++++++++++ .../aggregate/aggregateOperators.scala | 175 ----- .../aggregate/sortBasedIterators.scala | 664 ------------------- .../spark/sql/execution/aggregate/udaf.scala | 269 +++++++- .../spark/sql/execution/aggregate/utils.scala | 99 +-- .../spark/sql/execution/basicOperators.scala | 1 - .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- .../execution/SparkSqlSerializer2Suite.scala | 9 +- .../hive/execution/AggregationQuerySuite.scala | 118 ++-- 13 files changed, 1697 insertions(+), 973 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala ---------------------------------------------------------------------- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index d08f553..4abfdfe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -110,7 +110,11 @@ abstract class AggregateFunction2 * buffer value of `avg(x)` will be 0 and the position of the first buffer value of `avg(y)` * will be 2. */ - var mutableBufferOffset: Int = 0 + protected var mutableBufferOffset: Int = 0 + + def withNewMutableBufferOffset(newMutableBufferOffset: Int): Unit = { + mutableBufferOffset = newMutableBufferOffset + } /** * The offset of this function's start buffer value in the @@ -126,7 +130,11 @@ abstract class AggregateFunction2 * buffer value of `avg(x)` will be 1 and the position of the first buffer value of `avg(y)` * will be 3 (position 0 is used for the value of key`). */ - var inputBufferOffset: Int = 0 + protected var inputBufferOffset: Int = 0 + + def withNewInputBufferOffset(newInputBufferOffset: Int): Unit = { + inputBufferOffset = newInputBufferOffset + } /** The schema of the aggregation buffer. */ def bufferSchema: StructType @@ -195,11 +203,8 @@ abstract class AlgebraicAggregate extends AggregateFunction2 with Serializable w override def bufferSchema: StructType = StructType.fromAttributes(bufferAttributes) override def initialize(buffer: MutableRow): Unit = { - var i = 0 - while (i < bufferAttributes.size) { - buffer(i + mutableBufferOffset) = initialValues(i).eval() - i += 1 - } + throw new UnsupportedOperationException( + "AlgebraicAggregate's initialize should not be called directly") } override final def update(buffer: MutableRow, input: InternalRow): Unit = { http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala new file mode 100644 index 0000000..cf568dc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/Aggregate.scala @@ -0,0 +1,182 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.aggregate + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples, Distribution} +import org.apache.spark.sql.execution.{UnsafeFixedWidthAggregationMap, SparkPlan, UnaryNode} +import org.apache.spark.sql.types.StructType + +/** + * An Aggregate Operator used to evaluate [[AggregateFunction2]]. Based on the data types + * of the grouping expressions and aggregate functions, it determines if it uses + * sort-based aggregation and hybrid (hash-based with sort-based as the fallback) to + * process input rows. + */ +case class Aggregate( + requiredChildDistributionExpressions: Option[Seq[Expression]], + groupingExpressions: Seq[NamedExpression], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + child: SparkPlan) + extends UnaryNode { + + private[this] val allAggregateExpressions = + nonCompleteAggregateExpressions ++ completeAggregateExpressions + + private[this] val hasNonAlgebricAggregateFunctions = + !allAggregateExpressions.forall(_.aggregateFunction.isInstanceOf[AlgebraicAggregate]) + + // Use the hybrid iterator if (1) unsafe is enabled, (2) the schemata of + // grouping key and aggregation buffer is supported; and (3) all + // aggregate functions are algebraic. + private[this] val supportsHybridIterator: Boolean = { + val aggregationBufferSchema: StructType = + StructType.fromAttributes( + allAggregateExpressions.flatMap(_.aggregateFunction.bufferAttributes)) + val groupKeySchema: StructType = + StructType.fromAttributes(groupingExpressions.map(_.toAttribute)) + + val schemaSupportsUnsafe: Boolean = + UnsafeFixedWidthAggregationMap.supportsAggregationBufferSchema(aggregationBufferSchema) && + UnsafeProjection.canSupport(groupKeySchema) + + // TODO: Use the hybrid iterator for non-algebric aggregate functions. + sqlContext.conf.unsafeEnabled && schemaSupportsUnsafe && !hasNonAlgebricAggregateFunctions + } + + // We need to use sorted input if we have grouping expressions, and + // we cannot use the hybrid iterator or the hybrid is disabled. + private[this] val requiresSortedInput: Boolean = { + groupingExpressions.nonEmpty && !supportsHybridIterator + } + + override def canProcessUnsafeRows: Boolean = !hasNonAlgebricAggregateFunctions + + // If result expressions' data types are all fixed length, we generate unsafe rows + // (We have this requirement instead of check the result of UnsafeProjection.canSupport + // is because we use a mutable projection to generate the result). + override def outputsUnsafeRows: Boolean = { + // resultExpressions.map(_.dataType).forall(UnsafeRow.isFixedLength) + // TODO: Supports generating UnsafeRows. We can just re-enable the line above and fix + // any issue we get. + false + } + + override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) + + override def requiredChildDistribution: List[Distribution] = { + requiredChildDistributionExpressions match { + case Some(exprs) if exprs.length == 0 => AllTuples :: Nil + case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: Nil + case None => UnspecifiedDistribution :: Nil + } + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = { + if (requiresSortedInput) { + // TODO: We should not sort the input rows if they are just in reversed order. + groupingExpressions.map(SortOrder(_, Ascending)) :: Nil + } else { + Seq.fill(children.size)(Nil) + } + } + + override def outputOrdering: Seq[SortOrder] = { + if (requiresSortedInput) { + // It is possible that the child.outputOrdering starts with the required + // ordering expressions (e.g. we require [a] as the sort expression and the + // child's outputOrdering is [a, b]). We can only guarantee the output rows + // are sorted by values of groupingExpressions. + groupingExpressions.map(SortOrder(_, Ascending)) + } else { + Nil + } + } + + protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { + child.execute().mapPartitions { iter => + // Because the constructor of an aggregation iterator will read at least the first row, + // we need to get the value of iter.hasNext first. + val hasInput = iter.hasNext + val useHybridIterator = + hasInput && + supportsHybridIterator && + groupingExpressions.nonEmpty + if (useHybridIterator) { + UnsafeHybridAggregationIterator.createFromInputIterator( + groupingExpressions, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection _, + child.output, + iter, + outputsUnsafeRows) + } else { + if (!hasInput && groupingExpressions.nonEmpty) { + // This is a grouped aggregate and the input iterator is empty, + // so return an empty iterator. + Iterator[InternalRow]() + } else { + val outputIter = SortBasedAggregationIterator.createFromInputIterator( + groupingExpressions, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection _ , + newProjection _, + child.output, + iter, + outputsUnsafeRows) + if (!hasInput && groupingExpressions.isEmpty) { + // There is no input and there is no grouping expressions. + // We need to output a single row as the output. + Iterator[InternalRow](outputIter.outputForEmptyGroupingKeyWithoutInput()) + } else { + outputIter + } + } + } + } + } + + override def simpleString: String = { + val iterator = if (supportsHybridIterator && groupingExpressions.nonEmpty) { + classOf[UnsafeHybridAggregationIterator].getSimpleName + } else { + classOf[SortBasedAggregationIterator].getSimpleName + } + + s"""NewAggregate with $iterator ${groupingExpressions} ${allAggregateExpressions}""" + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala new file mode 100644 index 0000000..abca373 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -0,0 +1,490 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.aggregate + +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.unsafe.KVIterator + +import scala.collection.mutable.ArrayBuffer + +/** + * The base class of [[SortBasedAggregationIterator]] and [[UnsafeHybridAggregationIterator]]. + * It mainly contains two parts: + * 1. It initializes aggregate functions. + * 2. It creates two functions, `processRow` and `generateOutput` based on [[AggregateMode]] of + * its aggregate functions. `processRow` is the function to handle an input. `generateOutput` + * is used to generate result. + */ +abstract class AggregationIterator( + groupingKeyAttributes: Seq[Attribute], + valueAttributes: Seq[Attribute], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + outputsUnsafeRows: Boolean) + extends Iterator[InternalRow] with Logging { + + /////////////////////////////////////////////////////////////////////////// + // Initializing functions. + /////////////////////////////////////////////////////////////////////////// + + // An Seq of all AggregateExpressions. + // It is important that all AggregateExpressions with the mode Partial, PartialMerge or Final + // are at the beginning of the allAggregateExpressions. + protected val allAggregateExpressions = + nonCompleteAggregateExpressions ++ completeAggregateExpressions + + require( + allAggregateExpressions.map(_.mode).distinct.length <= 2, + s"$allAggregateExpressions are not supported becuase they have more than 2 distinct modes.") + + /** + * The distinct modes of AggregateExpressions. Right now, we can handle the following mode: + * - Partial-only: all AggregateExpressions have the mode of Partial; + * - PartialMerge-only: all AggregateExpressions have the mode of PartialMerge); + * - Final-only: all AggregateExpressions have the mode of Final; + * - Final-Complete: some AggregateExpressions have the mode of Final and + * others have the mode of Complete; + * - Complete-only: nonCompleteAggregateExpressions is empty and we have AggregateExpressions + * with mode Complete in completeAggregateExpressions; and + * - Grouping-only: there is no AggregateExpression. + */ + protected val aggregationMode: (Option[AggregateMode], Option[AggregateMode]) = + nonCompleteAggregateExpressions.map(_.mode).distinct.headOption -> + completeAggregateExpressions.map(_.mode).distinct.headOption + + // Initialize all AggregateFunctions by binding references if necessary, + // and set inputBufferOffset and mutableBufferOffset. + protected val allAggregateFunctions: Array[AggregateFunction2] = { + var mutableBufferOffset = 0 + var inputBufferOffset: Int = initialInputBufferOffset + val functions = new Array[AggregateFunction2](allAggregateExpressions.length) + var i = 0 + while (i < allAggregateExpressions.length) { + val func = allAggregateExpressions(i).aggregateFunction + val funcWithBoundReferences = allAggregateExpressions(i).mode match { + case Partial | Complete if !func.isInstanceOf[AlgebraicAggregate] => + // We need to create BoundReferences if the function is not an + // AlgebraicAggregate (it does not support code-gen) and the mode of + // this function is Partial or Complete because we will call eval of this + // function's children in the update method of this aggregate function. + // Those eval calls require BoundReferences to work. + BindReferences.bindReference(func, valueAttributes) + case _ => + // We only need to set inputBufferOffset for aggregate functions with mode + // PartialMerge and Final. + func.withNewInputBufferOffset(inputBufferOffset) + inputBufferOffset += func.bufferSchema.length + func + } + // Set mutableBufferOffset for this function. It is important that setting + // mutableBufferOffset happens after all potential bindReference operations + // because bindReference will create a new instance of the function. + funcWithBoundReferences.withNewMutableBufferOffset(mutableBufferOffset) + mutableBufferOffset += funcWithBoundReferences.bufferSchema.length + functions(i) = funcWithBoundReferences + i += 1 + } + functions + } + + // Positions of those non-algebraic aggregate functions in allAggregateFunctions. + // For example, we have func1, func2, func3, func4 in aggregateFunctions, and + // func2 and func3 are non-algebraic aggregate functions. + // nonAlgebraicAggregateFunctionPositions will be [1, 2]. + private[this] val allNonAlgebraicAggregateFunctionPositions: Array[Int] = { + val positions = new ArrayBuffer[Int]() + var i = 0 + while (i < allAggregateFunctions.length) { + allAggregateFunctions(i) match { + case agg: AlgebraicAggregate => + case _ => positions += i + } + i += 1 + } + positions.toArray + } + + // All AggregateFunctions functions with mode Partial, PartialMerge, or Final. + private[this] val nonCompleteAggregateFunctions: Array[AggregateFunction2] = + allAggregateFunctions.take(nonCompleteAggregateExpressions.length) + + // All non-algebraic aggregate functions with mode Partial, PartialMerge, or Final. + private[this] val nonCompleteNonAlgebraicAggregateFunctions: Array[AggregateFunction2] = + nonCompleteAggregateFunctions.collect { + case func: AggregateFunction2 if !func.isInstanceOf[AlgebraicAggregate] => func + } + + // The projection used to initialize buffer values for all AlgebraicAggregates. + private[this] val algebraicInitialProjection = { + val initExpressions = allAggregateFunctions.flatMap { + case ae: AlgebraicAggregate => ae.initialValues + case agg: AggregateFunction2 => Seq.fill(agg.bufferAttributes.length)(NoOp) + } + newMutableProjection(initExpressions, Nil)() + } + + // All non-Algebraic AggregateFunctions. + private[this] val allNonAlgebraicAggregateFunctions = + allNonAlgebraicAggregateFunctionPositions.map(allAggregateFunctions) + + /////////////////////////////////////////////////////////////////////////// + // Methods and fields used by sub-classes. + /////////////////////////////////////////////////////////////////////////// + + // Initializing functions used to process a row. + protected val processRow: (MutableRow, InternalRow) => Unit = { + val rowToBeProcessed = new JoinedRow + val aggregationBufferSchema = allAggregateFunctions.flatMap(_.bufferAttributes) + aggregationMode match { + // Partial-only + case (Some(Partial), None) => + val updateExpressions = nonCompleteAggregateFunctions.flatMap { + case ae: AlgebraicAggregate => ae.updateExpressions + case agg: AggregateFunction2 => Seq.fill(agg.bufferAttributes.length)(NoOp) + } + val algebraicUpdateProjection = + newMutableProjection(updateExpressions, aggregationBufferSchema ++ valueAttributes)() + + (currentBuffer: MutableRow, row: InternalRow) => { + algebraicUpdateProjection.target(currentBuffer) + // Process all algebraic aggregate functions. + algebraicUpdateProjection(rowToBeProcessed(currentBuffer, row)) + // Process all non-algebraic aggregate functions. + var i = 0 + while (i < nonCompleteNonAlgebraicAggregateFunctions.length) { + nonCompleteNonAlgebraicAggregateFunctions(i).update(currentBuffer, row) + i += 1 + } + } + + // PartialMerge-only or Final-only + case (Some(PartialMerge), None) | (Some(Final), None) => + val inputAggregationBufferSchema = if (initialInputBufferOffset == 0) { + // If initialInputBufferOffset, the input value does not contain + // grouping keys. + // This part is pretty hacky. + allAggregateFunctions.flatMap(_.cloneBufferAttributes).toSeq + } else { + groupingKeyAttributes ++ allAggregateFunctions.flatMap(_.cloneBufferAttributes) + } + // val inputAggregationBufferSchema = + // groupingKeyAttributes ++ + // allAggregateFunctions.flatMap(_.cloneBufferAttributes) + val mergeExpressions = nonCompleteAggregateFunctions.flatMap { + case ae: AlgebraicAggregate => ae.mergeExpressions + case agg: AggregateFunction2 => Seq.fill(agg.bufferAttributes.length)(NoOp) + } + // This projection is used to merge buffer values for all AlgebraicAggregates. + val algebraicMergeProjection = + newMutableProjection( + mergeExpressions, + aggregationBufferSchema ++ inputAggregationBufferSchema)() + + (currentBuffer: MutableRow, row: InternalRow) => { + // Process all algebraic aggregate functions. + algebraicMergeProjection.target(currentBuffer)(rowToBeProcessed(currentBuffer, row)) + // Process all non-algebraic aggregate functions. + var i = 0 + while (i < nonCompleteNonAlgebraicAggregateFunctions.length) { + nonCompleteNonAlgebraicAggregateFunctions(i).merge(currentBuffer, row) + i += 1 + } + } + + // Final-Complete + case (Some(Final), Some(Complete)) => + val completeAggregateFunctions: Array[AggregateFunction2] = + allAggregateFunctions.takeRight(completeAggregateExpressions.length) + // All non-algebraic aggregate functions with mode Complete. + val completeNonAlgebraicAggregateFunctions: Array[AggregateFunction2] = + completeAggregateFunctions.collect { + case func: AggregateFunction2 if !func.isInstanceOf[AlgebraicAggregate] => func + } + + // The first initialInputBufferOffset values of the input aggregation buffer is + // for grouping expressions and distinct columns. + val groupingAttributesAndDistinctColumns = valueAttributes.take(initialInputBufferOffset) + + val completeOffsetExpressions = + Seq.fill(completeAggregateFunctions.map(_.bufferAttributes.length).sum)(NoOp) + // We do not touch buffer values of aggregate functions with the Final mode. + val finalOffsetExpressions = + Seq.fill(nonCompleteAggregateFunctions.map(_.bufferAttributes.length).sum)(NoOp) + + val mergeInputSchema = + aggregationBufferSchema ++ + groupingAttributesAndDistinctColumns ++ + nonCompleteAggregateFunctions.flatMap(_.cloneBufferAttributes) + val mergeExpressions = + nonCompleteAggregateFunctions.flatMap { + case ae: AlgebraicAggregate => ae.mergeExpressions + case agg: AggregateFunction2 => Seq.fill(agg.bufferAttributes.length)(NoOp) + } ++ completeOffsetExpressions + val finalAlgebraicMergeProjection = + newMutableProjection(mergeExpressions, mergeInputSchema)() + + val updateExpressions = + finalOffsetExpressions ++ completeAggregateFunctions.flatMap { + case ae: AlgebraicAggregate => ae.updateExpressions + case agg: AggregateFunction2 => Seq.fill(agg.bufferAttributes.length)(NoOp) + } + val completeAlgebraicUpdateProjection = + newMutableProjection(updateExpressions, aggregationBufferSchema ++ valueAttributes)() + + (currentBuffer: MutableRow, row: InternalRow) => { + val input = rowToBeProcessed(currentBuffer, row) + // For all aggregate functions with mode Complete, update buffers. + completeAlgebraicUpdateProjection.target(currentBuffer)(input) + var i = 0 + while (i < completeNonAlgebraicAggregateFunctions.length) { + completeNonAlgebraicAggregateFunctions(i).update(currentBuffer, row) + i += 1 + } + + // For all aggregate functions with mode Final, merge buffers. + finalAlgebraicMergeProjection.target(currentBuffer)(input) + i = 0 + while (i < nonCompleteNonAlgebraicAggregateFunctions.length) { + nonCompleteNonAlgebraicAggregateFunctions(i).merge(currentBuffer, row) + i += 1 + } + } + + // Complete-only + case (None, Some(Complete)) => + val completeAggregateFunctions: Array[AggregateFunction2] = + allAggregateFunctions.takeRight(completeAggregateExpressions.length) + // All non-algebraic aggregate functions with mode Complete. + val completeNonAlgebraicAggregateFunctions: Array[AggregateFunction2] = + completeAggregateFunctions.collect { + case func: AggregateFunction2 if !func.isInstanceOf[AlgebraicAggregate] => func + } + + val updateExpressions = + completeAggregateFunctions.flatMap { + case ae: AlgebraicAggregate => ae.updateExpressions + case agg: AggregateFunction2 => Seq.fill(agg.bufferAttributes.length)(NoOp) + } + val completeAlgebraicUpdateProjection = + newMutableProjection(updateExpressions, aggregationBufferSchema ++ valueAttributes)() + + (currentBuffer: MutableRow, row: InternalRow) => { + val input = rowToBeProcessed(currentBuffer, row) + // For all aggregate functions with mode Complete, update buffers. + completeAlgebraicUpdateProjection.target(currentBuffer)(input) + var i = 0 + while (i < completeNonAlgebraicAggregateFunctions.length) { + completeNonAlgebraicAggregateFunctions(i).update(currentBuffer, row) + i += 1 + } + } + + // Grouping only. + case (None, None) => (currentBuffer: MutableRow, row: InternalRow) => {} + + case other => + sys.error( + s"Could not evaluate ${nonCompleteAggregateExpressions} because we do not " + + s"support evaluate modes $other in this iterator.") + } + } + + // Initializing the function used to generate the output row. + protected val generateOutput: (InternalRow, MutableRow) => InternalRow = { + val rowToBeEvaluated = new JoinedRow + val safeOutoutRow = new GenericMutableRow(resultExpressions.length) + val mutableOutput = if (outputsUnsafeRows) { + UnsafeProjection.create(resultExpressions.map(_.dataType).toArray).apply(safeOutoutRow) + } else { + safeOutoutRow + } + + aggregationMode match { + // Partial-only or PartialMerge-only: every output row is basically the values of + // the grouping expressions and the corresponding aggregation buffer. + case (Some(Partial), None) | (Some(PartialMerge), None) => + // Because we cannot copy a joinedRow containing a UnsafeRow (UnsafeRow does not + // support generic getter), we create a mutable projection to output the + // JoinedRow(currentGroupingKey, currentBuffer) + val bufferSchema = nonCompleteAggregateFunctions.flatMap(_.bufferAttributes) + val resultProjection = + newMutableProjection( + groupingKeyAttributes ++ bufferSchema, + groupingKeyAttributes ++ bufferSchema)() + resultProjection.target(mutableOutput) + + (currentGroupingKey: InternalRow, currentBuffer: MutableRow) => { + resultProjection(rowToBeEvaluated(currentGroupingKey, currentBuffer)) + // rowToBeEvaluated(currentGroupingKey, currentBuffer) + } + + // Final-only, Complete-only and Final-Complete: every output row contains values representing + // resultExpressions. + case (Some(Final), None) | (Some(Final) | None, Some(Complete)) => + val bufferSchemata = + allAggregateFunctions.flatMap(_.bufferAttributes) + val evalExpressions = allAggregateFunctions.map { + case ae: AlgebraicAggregate => ae.evaluateExpression + case agg: AggregateFunction2 => NoOp + } + val algebraicEvalProjection = newMutableProjection(evalExpressions, bufferSchemata)() + val aggregateResultSchema = nonCompleteAggregateAttributes ++ completeAggregateAttributes + // TODO: Use unsafe row. + val aggregateResult = new GenericMutableRow(aggregateResultSchema.length) + val resultProjection = + newMutableProjection( + resultExpressions, groupingKeyAttributes ++ aggregateResultSchema)() + resultProjection.target(mutableOutput) + + (currentGroupingKey: InternalRow, currentBuffer: MutableRow) => { + // Generate results for all algebraic aggregate functions. + algebraicEvalProjection.target(aggregateResult)(currentBuffer) + // Generate results for all non-algebraic aggregate functions. + var i = 0 + while (i < allNonAlgebraicAggregateFunctions.length) { + aggregateResult.update( + allNonAlgebraicAggregateFunctionPositions(i), + allNonAlgebraicAggregateFunctions(i).eval(currentBuffer)) + i += 1 + } + resultProjection(rowToBeEvaluated(currentGroupingKey, aggregateResult)) + } + + // Grouping-only: we only output values of grouping expressions. + case (None, None) => + val resultProjection = + newMutableProjection(resultExpressions, groupingKeyAttributes)() + resultProjection.target(mutableOutput) + + (currentGroupingKey: InternalRow, currentBuffer: MutableRow) => { + resultProjection(currentGroupingKey) + } + + case other => + sys.error( + s"Could not evaluate ${nonCompleteAggregateExpressions} because we do not " + + s"support evaluate modes $other in this iterator.") + } + } + + /** Initializes buffer values for all aggregate functions. */ + protected def initializeBuffer(buffer: MutableRow): Unit = { + algebraicInitialProjection.target(buffer)(EmptyRow) + var i = 0 + while (i < allNonAlgebraicAggregateFunctions.length) { + allNonAlgebraicAggregateFunctions(i).initialize(buffer) + i += 1 + } + } + + /** + * Creates a new aggregation buffer and initializes buffer values + * for all aggregate functions. + */ + protected def newBuffer: MutableRow +} + +object AggregationIterator { + def kvIterator( + groupingExpressions: Seq[NamedExpression], + newProjection: (Seq[Expression], Seq[Attribute]) => Projection, + inputAttributes: Seq[Attribute], + inputIter: Iterator[InternalRow]): KVIterator[InternalRow, InternalRow] = { + new KVIterator[InternalRow, InternalRow] { + private[this] val groupingKeyGenerator = newProjection(groupingExpressions, inputAttributes) + + private[this] var groupingKey: InternalRow = _ + + private[this] var value: InternalRow = _ + + override def next(): Boolean = { + if (inputIter.hasNext) { + // Read the next input row. + val inputRow = inputIter.next() + // Get groupingKey based on groupingExpressions. + groupingKey = groupingKeyGenerator(inputRow) + // The value is the inputRow. + value = inputRow + true + } else { + false + } + } + + override def getKey(): InternalRow = { + groupingKey + } + + override def getValue(): InternalRow = { + value + } + + override def close(): Unit = { + // Do nothing + } + } + } + + def unsafeKVIterator( + groupingExpressions: Seq[NamedExpression], + inputAttributes: Seq[Attribute], + inputIter: Iterator[InternalRow]): KVIterator[UnsafeRow, InternalRow] = { + new KVIterator[UnsafeRow, InternalRow] { + private[this] val groupingKeyGenerator = + UnsafeProjection.create(groupingExpressions, inputAttributes) + + private[this] var groupingKey: UnsafeRow = _ + + private[this] var value: InternalRow = _ + + override def next(): Boolean = { + if (inputIter.hasNext) { + // Read the next input row. + val inputRow = inputIter.next() + // Get groupingKey based on groupingExpressions. + groupingKey = groupingKeyGenerator.apply(inputRow) + // The value is the inputRow. + value = inputRow + true + } else { + false + } + } + + override def getKey(): UnsafeRow = { + groupingKey + } + + override def getValue(): InternalRow = { + value + } + + override def close(): Unit = { + // Do nothing + } + } + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala new file mode 100644 index 0000000..78bcee1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortBasedAggregationIterator.scala @@ -0,0 +1,236 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.aggregate + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression2, AggregateFunction2} +import org.apache.spark.sql.execution.UnsafeFixedWidthAggregationMap +import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.KVIterator + +/** + * An iterator used to evaluate [[AggregateFunction2]]. It assumes the input rows have been + * sorted by values of [[groupingKeyAttributes]]. + */ +class SortBasedAggregationIterator( + groupingKeyAttributes: Seq[Attribute], + valueAttributes: Seq[Attribute], + inputKVIterator: KVIterator[InternalRow, InternalRow], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + outputsUnsafeRows: Boolean) + extends AggregationIterator( + groupingKeyAttributes, + valueAttributes, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection, + outputsUnsafeRows) { + + override protected def newBuffer: MutableRow = { + val bufferSchema = allAggregateFunctions.flatMap(_.bufferAttributes) + val bufferRowSize: Int = bufferSchema.length + + val genericMutableBuffer = new GenericMutableRow(bufferRowSize) + val useUnsafeBuffer = bufferSchema.map(_.dataType).forall(UnsafeRow.isFixedLength) + + val buffer = if (useUnsafeBuffer) { + val unsafeProjection = + UnsafeProjection.create(bufferSchema.map(_.dataType)) + unsafeProjection.apply(genericMutableBuffer) + } else { + genericMutableBuffer + } + initializeBuffer(buffer) + buffer + } + + /////////////////////////////////////////////////////////////////////////// + // Mutable states for sort based aggregation. + /////////////////////////////////////////////////////////////////////////// + + // The partition key of the current partition. + private[this] var currentGroupingKey: InternalRow = _ + + // The partition key of next partition. + private[this] var nextGroupingKey: InternalRow = _ + + // The first row of next partition. + private[this] var firstRowInNextGroup: InternalRow = _ + + // Indicates if we has new group of rows from the sorted input iterator + private[this] var sortedInputHasNewGroup: Boolean = false + + // The aggregation buffer used by the sort-based aggregation. + private[this] val sortBasedAggregationBuffer: MutableRow = newBuffer + + /** Processes rows in the current group. It will stop when it find a new group. */ + protected def processCurrentSortedGroup(): Unit = { + currentGroupingKey = nextGroupingKey + // Now, we will start to find all rows belonging to this group. + // We create a variable to track if we see the next group. + var findNextPartition = false + // firstRowInNextGroup is the first row of this group. We first process it. + processRow(sortBasedAggregationBuffer, firstRowInNextGroup) + + // The search will stop when we see the next group or there is no + // input row left in the iter. + var hasNext = inputKVIterator.next() + while (!findNextPartition && hasNext) { + // Get the grouping key. + val groupingKey = inputKVIterator.getKey + val currentRow = inputKVIterator.getValue + + // Check if the current row belongs the current input row. + if (currentGroupingKey == groupingKey) { + processRow(sortBasedAggregationBuffer, currentRow) + + hasNext = inputKVIterator.next() + } else { + // We find a new group. + findNextPartition = true + nextGroupingKey = groupingKey.copy() + firstRowInNextGroup = currentRow.copy() + } + } + // We have not seen a new group. It means that there is no new row in the input + // iter. The current group is the last group of the iter. + if (!findNextPartition) { + sortedInputHasNewGroup = false + } + } + + /////////////////////////////////////////////////////////////////////////// + // Iterator's public methods + /////////////////////////////////////////////////////////////////////////// + + override final def hasNext: Boolean = sortedInputHasNewGroup + + override final def next(): InternalRow = { + if (hasNext) { + // Process the current group. + processCurrentSortedGroup() + // Generate output row for the current group. + val outputRow = generateOutput(currentGroupingKey, sortBasedAggregationBuffer) + // Initialize buffer values for the next group. + initializeBuffer(sortBasedAggregationBuffer) + + outputRow + } else { + // no more result + throw new NoSuchElementException + } + } + + protected def initialize(): Unit = { + if (inputKVIterator.next()) { + initializeBuffer(sortBasedAggregationBuffer) + + nextGroupingKey = inputKVIterator.getKey().copy() + firstRowInNextGroup = inputKVIterator.getValue().copy() + + sortedInputHasNewGroup = true + } else { + // This inputIter is empty. + sortedInputHasNewGroup = false + } + } + + initialize() + + def outputForEmptyGroupingKeyWithoutInput(): InternalRow = { + initializeBuffer(sortBasedAggregationBuffer) + generateOutput(new GenericInternalRow(0), sortBasedAggregationBuffer) + } +} + +object SortBasedAggregationIterator { + // scalastyle:off + def createFromInputIterator( + groupingExprs: Seq[NamedExpression], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + newProjection: (Seq[Expression], Seq[Attribute]) => Projection, + inputAttributes: Seq[Attribute], + inputIter: Iterator[InternalRow], + outputsUnsafeRows: Boolean): SortBasedAggregationIterator = { + val kvIterator = if (UnsafeProjection.canSupport(groupingExprs)) { + AggregationIterator.unsafeKVIterator( + groupingExprs, + inputAttributes, + inputIter).asInstanceOf[KVIterator[InternalRow, InternalRow]] + } else { + AggregationIterator.kvIterator(groupingExprs, newProjection, inputAttributes, inputIter) + } + + new SortBasedAggregationIterator( + groupingExprs.map(_.toAttribute), + inputAttributes, + kvIterator, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection, + outputsUnsafeRows) + } + + def createFromKVIterator( + groupingKeyAttributes: Seq[Attribute], + valueAttributes: Seq[Attribute], + inputKVIterator: KVIterator[InternalRow, InternalRow], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + outputsUnsafeRows: Boolean): SortBasedAggregationIterator = { + new SortBasedAggregationIterator( + groupingKeyAttributes, + valueAttributes, + inputKVIterator, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection, + outputsUnsafeRows) + } + // scalastyle:on +} http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala new file mode 100644 index 0000000..37d34eb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/UnsafeHybridAggregationIterator.scala @@ -0,0 +1,398 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.aggregate + +import org.apache.spark.sql.execution.{UnsafeKeyValueSorter, UnsafeFixedWidthAggregationMap} +import org.apache.spark.unsafe.KVIterator +import org.apache.spark.{SparkEnv, TaskContext} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.types.StructType + +/** + * An iterator used to evaluate [[AggregateFunction2]]. + * It first tries to use in-memory hash-based aggregation. If we cannot allocate more + * space for the hash map, we spill the sorted map entries, free the map, and then + * switch to sort-based aggregation. + */ +class UnsafeHybridAggregationIterator( + groupingKeyAttributes: Seq[Attribute], + valueAttributes: Seq[Attribute], + inputKVIterator: KVIterator[UnsafeRow, InternalRow], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + outputsUnsafeRows: Boolean) + extends AggregationIterator( + groupingKeyAttributes, + valueAttributes, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection, + outputsUnsafeRows) { + + require(groupingKeyAttributes.nonEmpty) + + /////////////////////////////////////////////////////////////////////////// + // Unsafe Aggregation buffers + /////////////////////////////////////////////////////////////////////////// + + // This is the Unsafe Aggregation Map used to store all buffers. + private[this] val buffers = new UnsafeFixedWidthAggregationMap( + newBuffer, + StructType.fromAttributes(allAggregateFunctions.flatMap(_.bufferAttributes)), + StructType.fromAttributes(groupingKeyAttributes), + TaskContext.get.taskMemoryManager(), + SparkEnv.get.shuffleMemoryManager, + 1024 * 16, // initial capacity + SparkEnv.get.conf.getSizeAsBytes("spark.buffer.pageSize", "64m"), + false // disable tracking of performance metrics + ) + + override protected def newBuffer: UnsafeRow = { + val bufferSchema = allAggregateFunctions.flatMap(_.bufferAttributes) + val bufferRowSize: Int = bufferSchema.length + + val genericMutableBuffer = new GenericMutableRow(bufferRowSize) + val unsafeProjection = + UnsafeProjection.create(bufferSchema.map(_.dataType)) + val buffer = unsafeProjection.apply(genericMutableBuffer) + initializeBuffer(buffer) + buffer + } + + /////////////////////////////////////////////////////////////////////////// + // Methods and variables related to switching to sort-based aggregation + /////////////////////////////////////////////////////////////////////////// + private[this] var sortBased = false + + private[this] var sortBasedAggregationIterator: SortBasedAggregationIterator = _ + + // The value part of the input KV iterator is used to store original input values of + // aggregate functions, we need to convert them to aggregation buffers. + private def processOriginalInput( + firstKey: UnsafeRow, + firstValue: InternalRow): KVIterator[UnsafeRow, UnsafeRow] = { + new KVIterator[UnsafeRow, UnsafeRow] { + private[this] var isFirstRow = true + + private[this] var groupingKey: UnsafeRow = _ + + private[this] val buffer: UnsafeRow = newBuffer + + override def next(): Boolean = { + initializeBuffer(buffer) + if (isFirstRow) { + isFirstRow = false + groupingKey = firstKey + processRow(buffer, firstValue) + + true + } else if (inputKVIterator.next()) { + groupingKey = inputKVIterator.getKey() + val value = inputKVIterator.getValue() + processRow(buffer, value) + + true + } else { + false + } + } + + override def getKey(): UnsafeRow = { + groupingKey + } + + override def getValue(): UnsafeRow = { + buffer + } + + override def close(): Unit = { + // Do nothing. + } + } + } + + // The value of the input KV Iterator has the format of groupingExprs + aggregation buffer. + // We need to project the aggregation buffer out. + private def projectInputBufferToUnsafe( + firstKey: UnsafeRow, + firstValue: InternalRow): KVIterator[UnsafeRow, UnsafeRow] = { + new KVIterator[UnsafeRow, UnsafeRow] { + private[this] var isFirstRow = true + + private[this] var groupingKey: UnsafeRow = _ + + private[this] val bufferSchema = allAggregateFunctions.flatMap(_.bufferAttributes) + + private[this] val value: UnsafeRow = { + val genericMutableRow = new GenericMutableRow(bufferSchema.length) + UnsafeProjection.create(bufferSchema.map(_.dataType)).apply(genericMutableRow) + } + + private[this] val projectInputBuffer = { + newMutableProjection(bufferSchema, valueAttributes)().target(value) + } + + override def next(): Boolean = { + if (isFirstRow) { + isFirstRow = false + groupingKey = firstKey + projectInputBuffer(firstValue) + + true + } else if (inputKVIterator.next()) { + groupingKey = inputKVIterator.getKey() + projectInputBuffer(inputKVIterator.getValue()) + + true + } else { + false + } + } + + override def getKey(): UnsafeRow = { + groupingKey + } + + override def getValue(): UnsafeRow = { + value + } + + override def close(): Unit = { + // Do nothing. + } + } + } + + /** + * We need to fall back to sort based aggregation because we do not have enough memory + * for our in-memory hash map (i.e. `buffers`). + */ + private def switchToSortBasedAggregation( + currentGroupingKey: UnsafeRow, + currentRow: InternalRow): Unit = { + logInfo("falling back to sort based aggregation.") + + // Step 1: Get the ExternalSorter containing entries of the map. + val externalSorter = buffers.destructAndCreateExternalSorter() + + // Step 2: Free the memory used by the map. + buffers.free() + + // Step 3: If we have aggregate function with mode Partial or Complete, + // we need to process them to get aggregation buffer. + // So, later in the sort-based aggregation iterator, we can do merge. + // If aggregate functions are with mode Final and PartialMerge, + // we just need to project the aggregation buffer from the input. + val needsProcess = aggregationMode match { + case (Some(Partial), None) => true + case (None, Some(Complete)) => true + case (Some(Final), Some(Complete)) => true + case _ => false + } + + val processedIterator = if (needsProcess) { + processOriginalInput(currentGroupingKey, currentRow) + } else { + // The input value's format is groupingExprs + buffer. + // We need to project the buffer part out. + projectInputBufferToUnsafe(currentGroupingKey, currentRow) + } + + // Step 4: Redirect processedIterator to externalSorter. + while (processedIterator.next()) { + externalSorter.insertKV(processedIterator.getKey(), processedIterator.getValue()) + } + + // Step 5: Get the sorted iterator from the externalSorter. + val sortedKVIterator: KVIterator[UnsafeRow, UnsafeRow] = externalSorter.sortedIterator() + + // Step 6: We now create a SortBasedAggregationIterator based on sortedKVIterator. + // For a aggregate function with mode Partial, its mode in the SortBasedAggregationIterator + // will be PartialMerge. For a aggregate function with mode Complete, + // its mode in the SortBasedAggregationIterator will be Final. + val newNonCompleteAggregateExpressions = allAggregateExpressions.map { + case AggregateExpression2(func, Partial, isDistinct) => + AggregateExpression2(func, PartialMerge, isDistinct) + case AggregateExpression2(func, Complete, isDistinct) => + AggregateExpression2(func, Final, isDistinct) + case other => other + } + val newNonCompleteAggregateAttributes = + nonCompleteAggregateAttributes ++ completeAggregateAttributes + + val newValueAttributes = + allAggregateExpressions.flatMap(_.aggregateFunction.cloneBufferAttributes) + + sortBasedAggregationIterator = SortBasedAggregationIterator.createFromKVIterator( + groupingKeyAttributes = groupingKeyAttributes, + valueAttributes = newValueAttributes, + inputKVIterator = sortedKVIterator.asInstanceOf[KVIterator[InternalRow, InternalRow]], + nonCompleteAggregateExpressions = newNonCompleteAggregateExpressions, + nonCompleteAggregateAttributes = newNonCompleteAggregateAttributes, + completeAggregateExpressions = Nil, + completeAggregateAttributes = Nil, + initialInputBufferOffset = 0, + resultExpressions = resultExpressions, + newMutableProjection = newMutableProjection, + outputsUnsafeRows = outputsUnsafeRows) + } + + /////////////////////////////////////////////////////////////////////////// + // Methods used to initialize this iterator. + /////////////////////////////////////////////////////////////////////////// + + /** Starts to read input rows and falls back to sort-based aggregation if necessary. */ + protected def initialize(): Unit = { + var hasNext = inputKVIterator.next() + while (!sortBased && hasNext) { + val groupingKey = inputKVIterator.getKey() + val currentRow = inputKVIterator.getValue() + val buffer = buffers.getAggregationBuffer(groupingKey) + if (buffer == null) { + // buffer == null means that we could not allocate more memory. + // Now, we need to spill the map and switch to sort-based aggregation. + switchToSortBasedAggregation(groupingKey, currentRow) + sortBased = true + } else { + processRow(buffer, currentRow) + hasNext = inputKVIterator.next() + } + } + } + + // This is the starting point of this iterator. + initialize() + + // Creates the iterator for the Hash Aggregation Map after we have populated + // contents of that map. + private[this] val aggregationBufferMapIterator = buffers.iterator() + + private[this] var _mapIteratorHasNext = false + + // Pre-load the first key-value pair from the map to make hasNext idempotent. + if (!sortBased) { + _mapIteratorHasNext = aggregationBufferMapIterator.next() + // If the map is empty, we just free it. + if (!_mapIteratorHasNext) { + buffers.free() + } + } + + /////////////////////////////////////////////////////////////////////////// + // Iterator's public methods + /////////////////////////////////////////////////////////////////////////// + + override final def hasNext: Boolean = { + (sortBased && sortBasedAggregationIterator.hasNext) || (!sortBased && _mapIteratorHasNext) + } + + + override final def next(): InternalRow = { + if (hasNext) { + if (sortBased) { + sortBasedAggregationIterator.next() + } else { + // We did not fall back to the sort-based aggregation. + val result = + generateOutput( + aggregationBufferMapIterator.getKey, + aggregationBufferMapIterator.getValue) + // Pre-load next key-value pair form aggregationBufferMapIterator. + _mapIteratorHasNext = aggregationBufferMapIterator.next() + + if (!_mapIteratorHasNext) { + val resultCopy = result.copy() + buffers.free() + resultCopy + } else { + result + } + } + } else { + // no more result + throw new NoSuchElementException + } + } +} + +object UnsafeHybridAggregationIterator { + // scalastyle:off + def createFromInputIterator( + groupingExprs: Seq[NamedExpression], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + inputAttributes: Seq[Attribute], + inputIter: Iterator[InternalRow], + outputsUnsafeRows: Boolean): UnsafeHybridAggregationIterator = { + new UnsafeHybridAggregationIterator( + groupingExprs.map(_.toAttribute), + inputAttributes, + AggregationIterator.unsafeKVIterator(groupingExprs, inputAttributes, inputIter), + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection, + outputsUnsafeRows) + } + + def createFromKVIterator( + groupingKeyAttributes: Seq[Attribute], + valueAttributes: Seq[Attribute], + inputKVIterator: KVIterator[UnsafeRow, InternalRow], + nonCompleteAggregateExpressions: Seq[AggregateExpression2], + nonCompleteAggregateAttributes: Seq[Attribute], + completeAggregateExpressions: Seq[AggregateExpression2], + completeAggregateAttributes: Seq[Attribute], + initialInputBufferOffset: Int, + resultExpressions: Seq[NamedExpression], + newMutableProjection: (Seq[Expression], Seq[Attribute]) => (() => MutableProjection), + outputsUnsafeRows: Boolean): UnsafeHybridAggregationIterator = { + new UnsafeHybridAggregationIterator( + groupingKeyAttributes, + valueAttributes, + inputKVIterator, + nonCompleteAggregateExpressions, + nonCompleteAggregateAttributes, + completeAggregateExpressions, + completeAggregateAttributes, + initialInputBufferOffset, + resultExpressions, + newMutableProjection, + outputsUnsafeRows) + } + // scalastyle:on +} http://git-wip-us.apache.org/repos/asf/spark/blob/1ebd41b1/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala deleted file mode 100644 index 98538c4..0000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala +++ /dev/null @@ -1,175 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.aggregate - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, UnspecifiedDistribution} -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode} - -case class Aggregate2Sort( - requiredChildDistributionExpressions: Option[Seq[Expression]], - groupingExpressions: Seq[NamedExpression], - aggregateExpressions: Seq[AggregateExpression2], - aggregateAttributes: Seq[Attribute], - resultExpressions: Seq[NamedExpression], - child: SparkPlan) - extends UnaryNode { - - override def canProcessUnsafeRows: Boolean = true - - override def references: AttributeSet = { - val referencesInResults = - AttributeSet(resultExpressions.flatMap(_.references)) -- AttributeSet(aggregateAttributes) - - AttributeSet( - groupingExpressions.flatMap(_.references) ++ - aggregateExpressions.flatMap(_.references) ++ - referencesInResults) - } - - override def requiredChildDistribution: List[Distribution] = { - requiredChildDistributionExpressions match { - case Some(exprs) if exprs.length == 0 => AllTuples :: Nil - case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) :: Nil - case None => UnspecifiedDistribution :: Nil - } - } - - override def requiredChildOrdering: Seq[Seq[SortOrder]] = { - // TODO: We should not sort the input rows if they are just in reversed order. - groupingExpressions.map(SortOrder(_, Ascending)) :: Nil - } - - override def outputOrdering: Seq[SortOrder] = { - // It is possible that the child.outputOrdering starts with the required - // ordering expressions (e.g. we require [a] as the sort expression and the - // child's outputOrdering is [a, b]). We can only guarantee the output rows - // are sorted by values of groupingExpressions. - groupingExpressions.map(SortOrder(_, Ascending)) - } - - override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) - - protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { - child.execute().mapPartitions { iter => - if (aggregateExpressions.length == 0) { - new FinalSortAggregationIterator( - groupingExpressions, - Nil, - Nil, - resultExpressions, - newMutableProjection, - child.output, - iter) - } else { - val aggregationIterator: SortAggregationIterator = { - aggregateExpressions.map(_.mode).distinct.toList match { - case Partial :: Nil => - new PartialSortAggregationIterator( - groupingExpressions, - aggregateExpressions, - newMutableProjection, - child.output, - iter) - case PartialMerge :: Nil => - new PartialMergeSortAggregationIterator( - groupingExpressions, - aggregateExpressions, - newMutableProjection, - child.output, - iter) - case Final :: Nil => - new FinalSortAggregationIterator( - groupingExpressions, - aggregateExpressions, - aggregateAttributes, - resultExpressions, - newMutableProjection, - child.output, - iter) - case other => - sys.error( - s"Could not evaluate ${aggregateExpressions} because we do not support evaluate " + - s"modes $other in this operator.") - } - } - - aggregationIterator - } - } - } -} - -case class FinalAndCompleteAggregate2Sort( - previousGroupingExpressions: Seq[NamedExpression], - groupingExpressions: Seq[NamedExpression], - finalAggregateExpressions: Seq[AggregateExpression2], - finalAggregateAttributes: Seq[Attribute], - completeAggregateExpressions: Seq[AggregateExpression2], - completeAggregateAttributes: Seq[Attribute], - resultExpressions: Seq[NamedExpression], - child: SparkPlan) - extends UnaryNode { - override def references: AttributeSet = { - val referencesInResults = - AttributeSet(resultExpressions.flatMap(_.references)) -- - AttributeSet(finalAggregateExpressions) -- - AttributeSet(completeAggregateExpressions) - - AttributeSet( - groupingExpressions.flatMap(_.references) ++ - finalAggregateExpressions.flatMap(_.references) ++ - completeAggregateExpressions.flatMap(_.references) ++ - referencesInResults) - } - - override def requiredChildDistribution: List[Distribution] = { - if (groupingExpressions.isEmpty) { - AllTuples :: Nil - } else { - ClusteredDistribution(groupingExpressions) :: Nil - } - } - - override def requiredChildOrdering: Seq[Seq[SortOrder]] = - groupingExpressions.map(SortOrder(_, Ascending)) :: Nil - - override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) - - protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { - child.execute().mapPartitions { iter => - - new FinalAndCompleteSortAggregationIterator( - previousGroupingExpressions.length, - groupingExpressions, - finalAggregateExpressions, - finalAggregateAttributes, - completeAggregateExpressions, - completeAggregateAttributes, - resultExpressions, - newMutableProjection, - child.output, - iter) - } - } - -} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org