Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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 @@ -128,7 +128,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
RemoveRedundantAliases,
RemoveNoopOperators,
SimplifyExtractValueOps,
CombineConcats) ++
CombineConcats,
TransformBinaryComparison) ++
extendedOperatorOptimizationRules

val operatorOptimizationBatch: Seq[Batch] = {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -735,3 +735,119 @@ object CombineConcats extends Rule[LogicalPlan] {
flattenConcats(concat)
}
}


/**
* Transform binary comparison(such as =, >, <, >=, <=) in conditions to its equivalent form,
* leaving attributes alone in one side, so that we can push it down to parquet or others.
* For example, this rule can optimize
* {{{
* SELECT * FROM table WHERE i + 3 = 5
* }}}
* to
* {{{
* SELECT * FROM table WHERE i = 5 - 3
* }}}
* when i is Int or Long, and then other rules will further optimize it to
* {{{
* SELECT * FROM table WHERE i = 2
* }}}
*/
object TransformBinaryComparison extends Rule[LogicalPlan] with PredicateHelper {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case q: LogicalPlan => q transformExpressionsUp {
case e @ BinaryComparison(left: BinaryArithmetic, right: Literal)
if isDataTypeSafe(left.dataType) =>
transformLeft(e, left, right)
case e @ BinaryComparison(left: Literal, right: BinaryArithmetic)
if isDataTypeSafe(right.dataType) =>
transformRight(e, left, right)
}
}

private def transformLeft(bc: BinaryComparison, left: BinaryArithmetic, right: Literal)
: Expression = {
left match {
case Add(ar: AttributeReference, lit: Literal) if isOptSafe(Subtract(right, lit)) =>
bc.makeCopy(Array(ar, Subtract(right, lit)))
case Add(lit: Literal, ar: AttributeReference) if isOptSafe(Subtract(right, lit)) =>
bc.makeCopy(Array(ar, Subtract(right, lit)))
case Subtract(ar: AttributeReference, lit: Literal) if isOptSafe(Add(right, lit)) =>
bc.makeCopy(Array(ar, Add(right, lit)))
case Subtract(lit: Literal, ar: AttributeReference) if isOptSafe(Subtract(lit, right)) =>
bc.makeCopy(Array(Subtract(lit, right), ar))
case _ => bc
}
}

private def transformRight(bc: BinaryComparison, left: Literal, right: BinaryArithmetic)
: Expression = {
right match {
case Add(ar: AttributeReference, lit: Literal) if isOptSafe(Subtract(left, lit)) =>
bc.makeCopy(Array(Subtract(left, lit), ar))
case Add(lit: Literal, ar: AttributeReference) if isOptSafe(Subtract(left, lit)) =>
bc.makeCopy(Array(Subtract(left, lit), ar))
case Subtract(ar: AttributeReference, lit: Literal) if isOptSafe(Add(left, lit)) =>
bc.makeCopy(Array(Add(left, lit), ar))
case Subtract(lit: Literal, ar: AttributeReference) if isOptSafe(Subtract(lit, left)) =>
bc.makeCopy(Array(ar, Subtract(lit, left)))
case _ => bc
}
}

private def isDataTypeSafe(dataType: DataType): Boolean = dataType match {
case IntegerType | LongType => true
case _ => false
}

private def isOptSafe(e: BinaryArithmetic): Boolean = {
val leftVal = e.left.eval(EmptyRow)
val rightVal = e.right.eval(EmptyRow)

e match {
case Add(_: Literal, _: Literal) =>
e.dataType match {
case IntegerType =>
isAddSafe(leftVal, rightVal, Int.MinValue, Int.MaxValue)
case LongType =>
isAddSafe(leftVal, rightVal, Long.MinValue, Long.MaxValue)
case _ => false
}

case Subtract(_: Literal, _: Literal) =>
e.dataType match {
case IntegerType =>
isSubtractSafe(leftVal, rightVal, Int.MinValue, Int.MaxValue)
case LongType =>
isSubtractSafe(leftVal, rightVal, Long.MinValue, Long.MaxValue)
case _ => false
}

case _ => false
}
}

private def isAddSafe[T](left: Any, right: Any, minValue: T, maxValue: T)
(implicit num: Numeric[T]): Boolean = {
import num._
val leftVal = left.asInstanceOf[T]
val rightVal = right.asInstanceOf[T]
if (rightVal > zero) {
leftVal <= maxValue - rightVal
} else {
leftVal >= minValue - rightVal
}
}

private def isSubtractSafe[T](left: Any, right: Any, minValue: T, maxValue: T)
(implicit num: Numeric[T]): Boolean = {
import num._
val leftVal = left.asInstanceOf[T]
val rightVal = right.asInstanceOf[T]
if (rightVal > zero) {
leftVal >= minValue + rightVal
} else {
leftVal <= maxValue + rightVal
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,223 @@
/*
* 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

/**
* Unit tests for transform binary comparision in expressions.
*/
class TransformBinaryComparisonSuite extends PlanTest {

object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
Batch("TransformBinaryComparison", FixedPoint(10),
ConstantFolding,
TransformBinaryComparison) :: Nil
}

val testRelation = LocalRelation('a.int, 'b.long)

private val columnA = 'a
private val columnB = 'b

test("test of int: a + 2 = 8") {
val query = testRelation
.where(Add(columnA, Literal(2)) === Literal(8))

val correctAnswer = testRelation
.where(columnA === Literal(6)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int: a + 2 >= 8") {
val query = testRelation
.where(Add(columnA, Literal(2)) >= Literal(8))

val correctAnswer = testRelation
.where(columnA >= Literal(6)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int: a + 2 <= 8") {
val query = testRelation
.where(Add(columnA, Literal(2)) <= Literal(8))

val correctAnswer = testRelation
.where(columnA <= Literal(6)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int: a - 2 <= 8") {
val query = testRelation
.where(Subtract(columnA, Literal(2)) <= Literal(8))

val correctAnswer = testRelation
.where(columnA <= Literal(10)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int: 2 - a <= 8") {
val query = testRelation
.where(Subtract(Literal(2), columnA) <= Literal(8))

val correctAnswer = testRelation
.where(Literal(-6) <= columnA).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int: 2 - a >= 8") {
val query = testRelation
.where(Subtract(Literal(2), columnA) >= Literal(8))

val correctAnswer = testRelation
.where(Literal(-6) >= columnA).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int with overflow risk: a - 10 >= Int.MaxValue - 2") {
val query = testRelation
.where(Subtract(columnA, Literal(10)) >= Literal(Int.MaxValue - 2))

val correctAnswer = testRelation
.where(Subtract(columnA, Literal(10)) >= Literal(Int.MaxValue - 2)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int with overflow risk: 10 - a >= Int.MinValue") {
val query = testRelation
.where(Subtract(Literal(10), columnA) >= Literal(Int.MinValue))

val correctAnswer = testRelation
.where(Subtract(Literal(10), columnA) >= Literal(Int.MinValue)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of int with overflow risk: a + 10 <= Int.MinValue + 2") {
val query = testRelation
.where(Add(columnA, Literal(10)) <= Literal(Int.MinValue + 2))

val correctAnswer = testRelation
.where(Add(columnA, Literal(10)) <= Literal(Int.MinValue + 2)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long: b + 2L = 8L") {
val query = testRelation
.where(Add(columnB, Literal(2L)) === Literal(8L))

val correctAnswer = testRelation
.where(columnB === Literal(6L)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long: b + 2L >= 8L") {
val query = testRelation
.where(Add(columnB, Literal(2L)) >= Literal(8L))

val correctAnswer = testRelation
.where(columnB >= Literal(6L)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long: b + 2L <= 8L") {
val query = testRelation
.where(Add(columnB, Literal(2L)) <= Literal(8L))

val correctAnswer = testRelation
.where(columnB <= Literal(6L)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long: b - 2L <= 8L") {
val query = testRelation
.where(Subtract(columnB, Literal(2L)) <= Literal(8L))

val correctAnswer = testRelation
.where(columnB <= Literal(10L)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long: 2L - b <= 8L") {
val query = testRelation
.where(Subtract(Literal(2L), columnB) <= Literal(8))

val correctAnswer = testRelation
.where(Literal(-6L) <= columnB).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long: 2L - b >= 8L") {
val query = testRelation
.where(Subtract(Literal(2L), columnB) >= Literal(8))

val correctAnswer = testRelation
.where(Literal(-6L) >= columnB).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long with overflow risk: b - 10L >= Long.MaxValue - 2") {
val query = testRelation
.where(Subtract(columnB, Literal(10L)) >= Literal(Long.MaxValue - 2))

val correctAnswer = testRelation
.where(Subtract(columnB, Literal(10L)) >= Literal(Long.MaxValue - 2)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long with overflow risk: 10L - b >= Long.MinValue") {
val query = testRelation
.where(Subtract(Literal(10L), columnB) >= Literal(Long.MinValue))

val correctAnswer = testRelation
.where(Subtract(Literal(10L), columnB) >= Literal(Long.MinValue)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}

test("test of long with overflow risk: bL + 10 <= Long.MinValue + 2") {
val query = testRelation
.where(Add(columnB, Literal(10)) <= Literal(Long.MinValue + 2))

val correctAnswer = testRelation
.where(Add(columnB, Literal(10L)) <= Literal(Long.MinValue + 2)).analyze

comparePlans(Optimize.execute(query.analyze), correctAnswer)
}
}