Skip to content

Commit

Permalink
Many updates:
Browse files Browse the repository at this point in the history
1. Implementing COUNT, FIRST, LAST, MAX, MIN, and SUM based on the new interface.
2. Automatically fall back to old aggregation code path if we cannot evaluate the query using the new code path.
3. Refactoring.
  • Loading branch information
yhuai committed Jul 21, 2015
1 parent 85c9c4b commit dc96fd1
Show file tree
Hide file tree
Showing 18 changed files with 812 additions and 387 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@
package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions.aggregate2.{DistinctAggregateExpression1, Complete, AggregateExpression2, AggregateFunction2}
import org.apache.spark.sql.catalyst.expressions.aggregate.{Complete, AggregateExpression2, AggregateFunction2}
import org.apache.spark.sql.catalyst.{SimpleCatalystConf, CatalystConf}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
Expand Down Expand Up @@ -521,9 +521,21 @@ class Analyzer(
case u @ UnresolvedFunction(name, children, isDistinct) =>
withPosition(u) {
registry.lookupFunction(name, children) match {
// We get an aggregate function built based on AggregateFunction2 interface.
// So, we wrap it in AggregateExpression2.
case agg2: AggregateFunction2 => AggregateExpression2(agg2, Complete, isDistinct)
case agg1: AggregateExpression1 if isDistinct =>
DistinctAggregateExpression1(agg1)
// Currently, our old aggregate function interface supports SUM(DISTINCT ...)
// and COUTN(DISTINCT ...).
case sumDistinct: SumDistinct => sumDistinct
case countDistinct: CountDistinct => countDistinct
// DISTINCT is not meaningful with Max and Min.
case max: Max if isDistinct => max
case min: Min if isDistinct => min
// For other aggregate functions, DISTINCT keyword is not supported for now.
// Once we converted to the new code path, we will allow using DISTINCT keyword.
case other if isDistinct =>
failAnalysis(s"$name does not support DISTINCT keyword.")
// If it does not have DISTINCT keyword, we will return it as is.
case other => other
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.analysis

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate2.AggregateExpression2
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression2
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.types._

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,291 @@
/*
* 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.catalyst.expressions.aggregate

import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.types._

case class Average(child: Expression) extends AlgebraicAggregate {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

// Return data type.
override def dataType: DataType = resultType

// Expected input data type.
// TODO: Once we remove the old code path, we can use our analyzer to cast NullType
// to the default data type of the NumericType.
override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))

private val resultType = child.dataType match {
case DecimalType.Fixed(precision, scale) =>
DecimalType(precision + 4, scale + 4)
case DecimalType.Unlimited => DecimalType.Unlimited
case _ => DoubleType
}

private val sumDataType = child.dataType match {
case _ @ DecimalType() => DecimalType.Unlimited
case _ => DoubleType
}

private val currentSum = AttributeReference("currentSum", sumDataType)()
private val currentCount = AttributeReference("currentCount", LongType)()

override val bufferAttributes = currentSum :: currentCount :: Nil

override val initialValues = Seq(
/* currentSum = */ Cast(Literal(0), sumDataType),
/* currentCount = */ Literal(0L)
)

override val updateExpressions = Seq(
/* currentSum = */
Add(
currentSum,
Coalesce(Cast(child, sumDataType) :: Cast(Literal(0), sumDataType) :: Nil)),
/* currentCount = */ If(IsNull(child), currentCount, currentCount + 1L)
)

override val mergeExpressions = Seq(
/* currentSum = */ currentSum.left + currentSum.right,
/* currentCount = */ currentCount.left + currentCount.right
)

// If all input are nulls, currentCount will be 0 and we will get null after the division.
override val evaluateExpression = Cast(currentSum, resultType) / Cast(currentCount, resultType)
}

case class Count(child: Expression) extends AlgebraicAggregate {
override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = false

// Return data type.
override def dataType: DataType = LongType

// Expected input data type.
override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)

private val currentCount = AttributeReference("currentCount", LongType)()

override val bufferAttributes = currentCount :: Nil

override val initialValues = Seq(
/* currentCount = */ Literal(0L)
)

override val updateExpressions = Seq(
/* currentCount = */ If(IsNull(child), currentCount, currentCount + 1L)
)

override val mergeExpressions = Seq(
/* currentCount = */ currentCount.left + currentCount.right
)

override val evaluateExpression = Cast(currentCount, LongType)
}

case class First(child: Expression) extends AlgebraicAggregate {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

// First is not a deterministic function.
override def deterministic: Boolean = false

// Return data type.
override def dataType: DataType = child.dataType

// Expected input data type.
override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)

private val first = AttributeReference("first", child.dataType)()

override val bufferAttributes = first :: Nil

override val initialValues = Seq(
/* first = */ Literal.create(null, child.dataType)
)

override val updateExpressions = Seq(
/* first = */ If(IsNull(first), child, first)
)

override val mergeExpressions = Seq(
/* first = */ If(IsNull(first.left), first.right, first.left)
)

override val evaluateExpression = first
}

case class Last(child: Expression) extends AlgebraicAggregate {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

// Last is not a deterministic function.
override def deterministic: Boolean = false

// Return data type.
override def dataType: DataType = child.dataType

// Expected input data type.
override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)

private val last = AttributeReference("last", child.dataType)()

override val bufferAttributes = last :: Nil

override val initialValues = Seq(
/* last = */ Literal.create(null, child.dataType)
)

override val updateExpressions = Seq(
/* last = */ If(IsNull(child), last, child)
)

override val mergeExpressions = Seq(
/* last = */ If(IsNull(last.right), last.left, last.right)
)

override val evaluateExpression = last
}

case class Max(child: Expression) extends AlgebraicAggregate {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

// Return data type.
override def dataType: DataType = child.dataType

// Expected input data type.
override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)

private val max = AttributeReference("max", child.dataType)()

override val bufferAttributes = max :: Nil

override val initialValues = Seq(
/* max = */ Literal.create(null, child.dataType)
)

override val updateExpressions = Seq(
/* max = */ If(IsNull(child), max, If(IsNull(max), child, Greatest(Seq(max, child))))
)

override val mergeExpressions = {
val greatest = Greatest(Seq(max.left, max.right))
Seq(
/* max = */ If(IsNull(max.right), max.left, If(IsNull(max.left), max.right, greatest))
)
}

override val evaluateExpression = max
}

case class Min(child: Expression) extends AlgebraicAggregate {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

// Return data type.
override def dataType: DataType = child.dataType

// Expected input data type.
override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)

private val min = AttributeReference("min", child.dataType)()

override val bufferAttributes = min :: Nil

override val initialValues = Seq(
/* min = */ Literal.create(null, child.dataType)
)

override val updateExpressions = Seq(
/* min = */ If(IsNull(child), min, If(IsNull(min), child, Least(Seq(min, child))))
)

override val mergeExpressions = {
val least = Least(Seq(min.left, min.right))
Seq(
/* min = */ If(IsNull(min.right), min.left, If(IsNull(min.left), min.right, least))
)
}

override val evaluateExpression = min
}

case class Sum(child: Expression) extends AlgebraicAggregate {

override def children: Seq[Expression] = child :: Nil

override def nullable: Boolean = true

// Return data type.
override def dataType: DataType = resultType

// Expected input data type.
override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(NumericType, NullType))

private val resultType = child.dataType match {
case DecimalType.Fixed(precision, scale) =>
DecimalType(precision + 4, scale + 4)
case DecimalType.Unlimited => DecimalType.Unlimited
case _ => DoubleType
}

private val sumDataType = child.dataType match {
case _ @ DecimalType() => DecimalType.Unlimited
case _ => DoubleType
}

private val currentSum = AttributeReference("currentSum", sumDataType)()

private val zero = Cast(Literal(0), sumDataType)

override val bufferAttributes = currentSum :: Nil

override val initialValues = Seq(
/* currentSum = */ Literal.create(null, sumDataType)
)

override val updateExpressions = Seq(
/* currentSum = */
Coalesce(Seq(Add(Coalesce(Seq(currentSum, zero)), Cast(child, sumDataType)), currentSum))
)

override val mergeExpressions = {
val add = Add(Coalesce(Seq(currentSum.left, zero)), Cast(currentSum.right, sumDataType))
Seq(
/* currentSum = */
Coalesce(Seq(add, currentSum.left))
)
}

override val evaluateExpression = Cast(currentSum, resultType)
}
Loading

0 comments on commit dc96fd1

Please sign in to comment.