Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-14664][SQL] Implement DecimalAggregates optimization for Window queries #12421

Closed
wants to merge 4 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -1343,17 +1343,35 @@ object DecimalAggregates extends Rule[LogicalPlan] {
/** Maximum number of decimal digits representable precisely in a Double */
private val MAX_DOUBLE_DIGITS = 15

def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case ae @ AggregateExpression(Sum(e @ DecimalType.Expression(prec, scale)), _, _, _)
if prec + 10 <= MAX_LONG_DIGITS =>
MakeDecimal(ae.copy(aggregateFunction = Sum(UnscaledValue(e))), prec + 10, scale)

case ae @ AggregateExpression(Average(e @ DecimalType.Expression(prec, scale)), _, _, _)
if prec + 4 <= MAX_DOUBLE_DIGITS =>
val newAggExpr = ae.copy(aggregateFunction = Average(UnscaledValue(e)))
Cast(
Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)),
DecimalType(prec + 4, scale + 4))
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

MINOR: I don't suppose there are ways to make this a bit more concise/readable?

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Indeed, those pattern matchings are too deep.
I'll remove some duplication caused by hierarchy.

case q: LogicalPlan => q transformExpressionsDown {
case we @ WindowExpression(ae @ AggregateExpression(af, _, _, _), _) => af match {
case Sum(e @ DecimalType.Expression(prec, scale)) if prec + 10 <= MAX_LONG_DIGITS =>
MakeDecimal(we.copy(windowFunction = ae.copy(aggregateFunction = Sum(UnscaledValue(e)))),
prec + 10, scale)

case Average(e @ DecimalType.Expression(prec, scale)) if prec + 4 <= MAX_DOUBLE_DIGITS =>
val newAggExpr =
we.copy(windowFunction = ae.copy(aggregateFunction = Average(UnscaledValue(e))))
Cast(
Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)),
DecimalType(prec + 4, scale + 4))

case _ => we
}
case ae @ AggregateExpression(af, _, _, _) => af match {
case Sum(e @ DecimalType.Expression(prec, scale)) if prec + 10 <= MAX_LONG_DIGITS =>
MakeDecimal(ae.copy(aggregateFunction = Sum(UnscaledValue(e))), prec + 10, scale)

case Average(e @ DecimalType.Expression(prec, scale)) if prec + 4 <= MAX_DOUBLE_DIGITS =>
val newAggExpr = ae.copy(aggregateFunction = Average(UnscaledValue(e)))
Cast(
Divide(newAggExpr, Literal.create(math.pow(10.0, scale), DoubleType)),
DecimalType(prec + 4, scale + 4))

case _ => ae
}
}
}
}

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

import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.types.DecimalType

class DecimalAggregatesSuite extends PlanTest {

object Optimize extends RuleExecutor[LogicalPlan] {
val batches = Batch("Decimal Optimizations", FixedPoint(100),
DecimalAggregates) :: Nil
}

val testRelation = LocalRelation('a.decimal(2, 1), 'b.decimal(12, 1))

test("Decimal Sum Aggregation: Optimized") {
val originalQuery = testRelation.select(sum('a))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = testRelation
.select(MakeDecimal(sum(UnscaledValue('a)), 12, 1).as("sum(a)")).analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Sum Aggregation: Not Optimized") {
val originalQuery = testRelation.select(sum('b))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = originalQuery.analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Average Aggregation: Optimized") {
val originalQuery = testRelation.select(avg('a))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = testRelation
.select((avg(UnscaledValue('a)) / 10.0).cast(DecimalType(6, 5)).as("avg(a)")).analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Average Aggregation: Not Optimized") {
val originalQuery = testRelation.select(avg('b))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = originalQuery.analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Sum Aggregation over Window: Optimized") {
val spec = windowSpec(Seq('a), Nil, UnspecifiedFrame)
val originalQuery = testRelation.select(windowExpr(sum('a), spec).as('sum_a))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = testRelation
.select('a)
.window(
Seq(MakeDecimal(windowExpr(sum(UnscaledValue('a)), spec), 12, 1).as('sum_a)),
Seq('a),
Nil)
.select('a, 'sum_a, 'sum_a)
.select('sum_a)
.analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Sum Aggregation over Window: Not Optimized") {
val spec = windowSpec('b :: Nil, Nil, UnspecifiedFrame)
val originalQuery = testRelation.select(windowExpr(sum('b), spec))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = originalQuery.analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Average Aggregation over Window: Optimized") {
val spec = windowSpec(Seq('a), Nil, UnspecifiedFrame)
val originalQuery = testRelation.select(windowExpr(avg('a), spec).as('avg_a))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = testRelation
.select('a)
.window(
Seq((windowExpr(avg(UnscaledValue('a)), spec) / 10.0).cast(DecimalType(6, 5)).as('avg_a)),
Seq('a),
Nil)
.select('a, 'avg_a, 'avg_a)
.select('avg_a)
.analyze

comparePlans(optimized, correctAnswer)
}

test("Decimal Average Aggregation over Window: Not Optimized") {
val spec = windowSpec('b :: Nil, Nil, UnspecifiedFrame)
val originalQuery = testRelation.select(windowExpr(avg('b), spec))
val optimized = Optimize.execute(originalQuery.analyze)
val correctAnswer = originalQuery.analyze

comparePlans(optimized, correctAnswer)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -22,7 +22,7 @@ import org.apache.spark.sql.functions._
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.test.SQLTestData.DecimalData
import org.apache.spark.sql.types.DecimalType
import org.apache.spark.sql.types.{Decimal, DecimalType}

case class Fact(date: Int, hour: Int, minute: Int, room_name: String, temp: Double)

Expand Down Expand Up @@ -430,4 +430,13 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext {
expr("kurtosis(a)")),
Row(null, null, null, null, null))
}

test("SPARK-14664: Decimal sum/avg over window should work.") {
checkAnswer(
sqlContext.sql("select sum(a) over () from values 1.0, 2.0, 3.0 T(a)"),
Row(6.0) :: Row(6.0) :: Row(6.0) :: Nil)
checkAnswer(
sqlContext.sql("select avg(a) over () from values 1.0, 2.0, 3.0 T(a)"),
Row(2.0) :: Row(2.0) :: Row(2.0) :: Nil)
}
}