Skip to content

Commit

Permalink
Bitwise operations are commutative
Browse files Browse the repository at this point in the history
  • Loading branch information
tanelk committed Sep 17, 2020
1 parent 2e3aa2f commit cf6c7e9
Show file tree
Hide file tree
Showing 2 changed files with 23 additions and 0 deletions.
Expand Up @@ -80,6 +80,13 @@ object Canonicalize {
orderCommutative(a, { case And(l, r) if l.deterministic && r.deterministic => Seq(l, r)})
.reduce(And)

case o: BitwiseOr =>
orderCommutative(o, { case BitwiseOr(l, r) => Seq(l, r) }).reduce(BitwiseOr)
case a: BitwiseAnd =>
orderCommutative(a, { case BitwiseAnd(l, r) => Seq(l, r) }).reduce(BitwiseAnd)
case x: BitwiseXor =>
orderCommutative(x, { case BitwiseXor(l, r) => Seq(l, r) }).reduce(BitwiseXor)

case EqualTo(l, r) if l.hashCode() > r.hashCode() => EqualTo(r, l)
case EqualNullSafe(l, r) if l.hashCode() > r.hashCode() => EqualNullSafe(r, l)

Expand Down
Expand Up @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions
import java.util.TimeZone

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.plans.logical.Range
import org.apache.spark.sql.types.{IntegerType, LongType, StructField, StructType}
Expand Down Expand Up @@ -95,4 +96,19 @@ class CanonicalizeSuite extends SparkFunSuite {
val castWithTimeZoneId = Cast(literal, LongType, Some(TimeZone.getDefault.getID))
assert(castWithTimeZoneId.semanticEquals(cast))
}

test("SPARK-32927: Bitwise operations are commutative") {
Seq(
(l: Expression, r: Expression) => BitwiseOr(l, r),
(l: Expression, r: Expression) => BitwiseAnd(l, r),
(l: Expression, r: Expression) => BitwiseXor(l, r)
).foreach(f => {
val e1 = f('a, f('b, 'c))
val e2 = f(f('a, 'b), 'c)
val e3 = f('a, f('b, 'a))

assert(e1.canonicalized == e2.canonicalized)
assert(e1.canonicalized != e3.canonicalized)
})
}
}

0 comments on commit cf6c7e9

Please sign in to comment.