diff options
author | Reynold Xin <rxin@apache.org> | 2014-06-20 00:34:59 -0700 |
---|---|---|
committer | Reynold Xin <rxin@apache.org> | 2014-06-20 00:34:59 -0700 |
commit | 2f6a835e1a039a0b1ba6e184b3350444b70f91df (patch) | |
tree | 417d16686b35559bfeae12f1f0b0049b87d62a72 /sql/hive | |
parent | 324952892085d1933bcf392ce8f2ced452fe741e (diff) | |
download | spark-2f6a835e1a039a0b1ba6e184b3350444b70f91df.tar.gz spark-2f6a835e1a039a0b1ba6e184b3350444b70f91df.tar.bz2 spark-2f6a835e1a039a0b1ba6e184b3350444b70f91df.zip |
[SPARK-2218] rename Equals to EqualTo in Spark SQL expressions.
Due to the existence of scala.Equals, it is very error prone to name the expression Equals, especially because we use a lot of partial functions and pattern matching in the optimizer.
Note that this sits on top of #1144.
Author: Reynold Xin <rxin@apache.org>
Closes #1146 from rxin/equals and squashes the following commits:
f8583fd [Reynold Xin] Merge branch 'master' of github.com:apache/spark into equals
326b388 [Reynold Xin] Merge branch 'master' of github.com:apache/spark into equals
bd19807 [Reynold Xin] Rename EqualsTo to EqualTo.
81148d1 [Reynold Xin] [SPARK-2218] rename Equals to EqualsTo in Spark SQL expressions.
c4e543d [Reynold Xin] [SPARK-2210] boolean cast on boolean value should be removed.
Diffstat (limited to 'sql/hive')
-rw-r--r-- | sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 10 | ||||
-rw-r--r-- | sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala | 8 |
2 files changed, 9 insertions, 9 deletions
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index df761b073a..ec653efcc8 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -698,7 +698,7 @@ private[hive] object HiveQl { val joinConditions = joinExpressions.sliding(2).map { case Seq(c1, c2) => - val predicates = (c1, c2).zipped.map { case (e1, e2) => Equals(e1, e2): Expression } + val predicates = (c1, c2).zipped.map { case (e1, e2) => EqualTo(e1, e2): Expression } predicates.reduceLeft(And) }.toBuffer @@ -924,9 +924,9 @@ private[hive] object HiveQl { case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) /* Comparisons */ - case Token("=", left :: right:: Nil) => Equals(nodeToExpr(left), nodeToExpr(right)) - case Token("!=", left :: right:: Nil) => Not(Equals(nodeToExpr(left), nodeToExpr(right))) - case Token("<>", left :: right:: Nil) => Not(Equals(nodeToExpr(left), nodeToExpr(right))) + case Token("=", left :: right:: Nil) => EqualTo(nodeToExpr(left), nodeToExpr(right)) + case Token("!=", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) + case Token("<>", left :: right:: Nil) => Not(EqualTo(nodeToExpr(left), nodeToExpr(right))) case Token(">", left :: right:: Nil) => GreaterThan(nodeToExpr(left), nodeToExpr(right)) case Token(">=", left :: right:: Nil) => GreaterThanOrEqual(nodeToExpr(left), nodeToExpr(right)) case Token("<", left :: right:: Nil) => LessThan(nodeToExpr(left), nodeToExpr(right)) @@ -966,7 +966,7 @@ private[hive] object HiveQl { // FIXME (SPARK-2155): the key will get evaluated for multiple times in CaseWhen's eval(). // Hence effectful / non-deterministic key expressions are *not* supported at the moment. // We should consider adding new Expressions to get around this. - Seq(Equals(nodeToExpr(branches(0)), nodeToExpr(condVal)), + Seq(EqualTo(nodeToExpr(branches(0)), nodeToExpr(condVal)), nodeToExpr(value)) case Seq(elseVal) => Seq(nodeToExpr(elseVal)) }.toSeq.reduce(_ ++ _) 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 cc8744c966..7436de264a 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,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.catalyst.expressions.{Cast, Equals} +import org.apache.spark.sql.catalyst.expressions.{Cast, EqualTo} import org.apache.spark.sql.execution.Project import org.apache.spark.sql.hive.test.TestHive @@ -39,13 +39,13 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { // No cast expression introduced project.transformAllExpressions { case c: Cast => - assert(false, "unexpected cast " + c) + fail(s"unexpected cast $c") c } - // Only one Equals + // Only one equality check var numEquals = 0 - project.transformAllExpressions { case e: Equals => + project.transformAllExpressions { case e: EqualTo => numEquals += 1 e } |