You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2014/06/20 08:58:26 UTC

git commit: [SPARK-2210] cast to boolean on boolean value gets turned into NOT((boolean_condition) = 0)

Repository: spark
Updated Branches:
  refs/heads/master f479cf374 -> 617564097


[SPARK-2210] cast to boolean on boolean value gets turned into NOT((boolean_condition) = 0)

```
explain select cast(cast(key=0 as boolean) as boolean) aaa from src
```
should be
```
[Physical execution plan:]
[Project [(key#10:0 = 0) AS aaa#7]]
[ HiveTableScan [key#10], (MetastoreRelation default, src, None), None]
```

However, it is currently
```
[Physical execution plan:]
[Project [NOT((key#10=0) = 0) AS aaa#7]]
[ HiveTableScan [key#10], (MetastoreRelation default, src, None), None]
```

Author: Reynold Xin <rx...@apache.org>

Closes #1144 from rxin/booleancast and squashes the following commits:

c4e543d [Reynold Xin] [SPARK-2210] boolean cast on boolean value should be removed.


Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/61756409
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/61756409
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/61756409

Branch: refs/heads/master
Commit: 61756409736a64bd42577782cb7468557fa0b642
Parents: f479cf3
Author: Reynold Xin <rx...@apache.org>
Authored: Thu Jun 19 23:58:23 2014 -0700
Committer: Reynold Xin <rx...@apache.org>
Committed: Thu Jun 19 23:58:23 2014 -0700

----------------------------------------------------------------------
 .../catalyst/analysis/HiveTypeCoercion.scala    |  4 +++-
 .../hive/execution/HiveTypeCoercionSuite.scala  | 25 +++++++++++++++++++-
 2 files changed, 27 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/61756409/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index 6d331fb..c0714bc 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -251,7 +251,9 @@ trait HiveTypeCoercion {
     def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
       // Skip nodes who's children have not been resolved yet.
       case e if !e.childrenResolved => e
-
+      // Skip if the type is boolean type already. Note that this extra cast should be removed
+      // by optimizer.SimplifyCasts.
+      case Cast(e, BooleanType) if e.dataType == BooleanType => e
       case Cast(e, BooleanType) => Not(Equals(e, Literal(0)))
       case Cast(e, dataType) if e.dataType == BooleanType =>
         Cast(If(e, Literal(1), Literal(0)), dataType)

http://git-wip-us.apache.org/repos/asf/spark/blob/61756409/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
index e030c8e..cc8744c 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
@@ -17,8 +17,12 @@
 
 package org.apache.spark.sql.hive.execution
 
+import org.apache.spark.sql.catalyst.expressions.{Cast, Equals}
+import org.apache.spark.sql.execution.Project
+import org.apache.spark.sql.hive.test.TestHive
+
 /**
- * A set of tests that validate type promotion rules.
+ * A set of tests that validate type promotion and coercion rules.
  */
 class HiveTypeCoercionSuite extends HiveComparisonTest {
   val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'")
@@ -28,4 +32,23 @@ class HiveTypeCoercionSuite extends HiveComparisonTest {
       createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1")
     }
   }
+
+  test("[SPARK-2210] boolean cast on boolean value should be removed") {
+    val q = "select cast(cast(key=0 as boolean) as boolean) from src"
+    val project = TestHive.hql(q).queryExecution.executedPlan.collect { case e: Project => e }.head
+
+    // No cast expression introduced
+    project.transformAllExpressions { case c: Cast =>
+      assert(false, "unexpected cast " + c)
+      c
+    }
+
+    // Only one Equals
+    var numEquals = 0
+    project.transformAllExpressions { case e: Equals =>
+      numEquals += 1
+      e
+    }
+    assert(numEquals === 1)
+  }
 }