aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorReynold Xin <rxin@apache.org>2014-06-20 00:34:59 -0700
committerReynold Xin <rxin@apache.org>2014-06-20 00:34:59 -0700
commit2f6a835e1a039a0b1ba6e184b3350444b70f91df (patch)
tree417d16686b35559bfeae12f1f0b0049b87d62a72 /sql
parent324952892085d1933bcf392ce8f2ced452fe741e (diff)
downloadspark-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')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala9
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala6
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala2
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala4
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala6
-rw-r--r--sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala4
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala4
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala19
-rw-r--r--sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala10
-rw-r--r--sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala8
11 files changed, 38 insertions, 40 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
index 2ad2d04af5..0cc4592047 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala
@@ -258,13 +258,13 @@ class SqlParser extends StandardTokenParsers with PackratParsers {
comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) })
protected lazy val comparisonExpression: Parser[Expression] =
- termExpression ~ "=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Equals(e1, e2) } |
+ termExpression ~ "=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => EqualTo(e1, e2) } |
termExpression ~ "<" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThan(e1, e2) } |
termExpression ~ "<=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => LessThanOrEqual(e1, e2) } |
termExpression ~ ">" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThan(e1, e2) } |
termExpression ~ ">=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => GreaterThanOrEqual(e1, e2) } |
- termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } |
- termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(Equals(e1, e2)) } |
+ termExpression ~ "!=" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } |
+ termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualTo(e1, e2)) } |
termExpression ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } |
termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } |
termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } |
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 c0714bcdd0..76ddeba9cb 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
@@ -234,8 +234,8 @@ 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
- // No need to change Equals operators as that actually makes sense for boolean types.
- case e: Equals => e
+ // No need to change EqualTo operators as that actually makes sense for boolean types.
+ case e: EqualTo => e
// Otherwise turn them to Byte types so that there exists and ordering.
case p: BinaryComparison
if p.left.dataType == BooleanType && p.right.dataType == BooleanType =>
@@ -254,7 +254,10 @@ trait HiveTypeCoercion {
// 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)))
+ // If the data type is not boolean and is being cast boolean, turn it into a comparison
+ // with the numeric value, i.e. x != 0. This will coerce the type into numeric type.
+ case Cast(e, BooleanType) if e.dataType != BooleanType => Not(EqualTo(e, Literal(0)))
+ // Turn true into 1, and false into 0 if casting boolean into other types.
case Cast(e, dataType) if e.dataType == BooleanType =>
Cast(If(e, Literal(1), Literal(0)), dataType)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index d177339d40..26ad4837b0 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -44,7 +44,7 @@ import org.apache.spark.sql.catalyst.types._
*
* // These unresolved attributes can be used to create more complicated expressions.
* scala> 'a === 'b
- * res2: org.apache.spark.sql.catalyst.expressions.Equals = ('a = 'b)
+ * res2: org.apache.spark.sql.catalyst.expressions.EqualTo = ('a = 'b)
*
* // SQL verbs can be used to construct logical query plans.
* scala> import org.apache.spark.sql.catalyst.plans.logical._
@@ -76,8 +76,8 @@ package object dsl {
def <= (other: Expression) = LessThanOrEqual(expr, other)
def > (other: Expression) = GreaterThan(expr, other)
def >= (other: Expression) = GreaterThanOrEqual(expr, other)
- def === (other: Expression) = Equals(expr, other)
- def !== (other: Expression) = Not(Equals(expr, other))
+ def === (other: Expression) = EqualTo(expr, other)
+ def !== (other: Expression) = Not(EqualTo(expr, other))
def like(other: Expression) = Like(expr, other)
def rlike(other: Expression) = RLike(expr, other)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
index 573ec052f4..b6f2451b52 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
@@ -24,7 +24,7 @@ package org.apache.spark.sql.catalyst
* expression, a [[NamedExpression]] in addition to the standard collection of expressions.
*
* ==Standard Expressions==
- * A library of standard expressions (e.g., [[Add]], [[Equals]]), aggregates (e.g., SUM, COUNT),
+ * A library of standard expressions (e.g., [[Add]], [[EqualTo]]), aggregates (e.g., SUM, COUNT),
* and other computations (e.g. UDFs). Each expression type is capable of determining its output
* schema as a function of its children's output schema.
*
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index 2718d43646..b63406b94a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -52,7 +52,7 @@ trait PredicateHelper {
*
* For example consider a join between two relations R(a, b) and S(c, d).
*
- * `canEvaluate(Equals(a,b), R)` returns `true` where as `canEvaluate(Equals(a,c), R)` returns
+ * `canEvaluate(EqualTo(a,b), R)` returns `true` where as `canEvaluate(EqualTo(a,c), R)` returns
* `false`.
*/
protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean =
@@ -140,7 +140,7 @@ abstract class BinaryComparison extends BinaryPredicate {
self: Product =>
}
-case class Equals(left: Expression, right: Expression) extends BinaryComparison {
+case class EqualTo(left: Expression, right: Expression) extends BinaryComparison {
def symbol = "="
override def eval(input: Row): Any = {
val l = left.eval(input)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index 820ecfb78b..a43bef389c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -136,14 +136,14 @@ object HashFilteredJoin extends Logging with PredicateHelper {
val Join(left, right, joinType, _) = join
val (joinPredicates, otherPredicates) =
allPredicates.flatMap(splitConjunctivePredicates).partition {
- case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) ||
+ case EqualTo(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) ||
(canEvaluate(l, right) && canEvaluate(r, left)) => true
case _ => false
}
val joinKeys = joinPredicates.map {
- case Equals(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r)
- case Equals(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l)
+ case EqualTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r)
+ case EqualTo(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => (r, l)
}
// Do not consider this strategy if there are no join keys.
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
index cea97c584f..0ff8206401 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
@@ -195,8 +195,8 @@ class ConstantFoldingSuite extends PlanTest {
Add(Literal(null, IntegerType), 1) as 'c9,
Add(1, Literal(null, IntegerType)) as 'c10,
- Equals(Literal(null, IntegerType), 1) as 'c11,
- Equals(1, Literal(null, IntegerType)) as 'c12,
+ EqualTo(Literal(null, IntegerType), 1) as 'c11,
+ EqualTo(1, Literal(null, IntegerType)) as 'c12,
Like(Literal(null, StringType), "abc") as 'c13,
Like("abc", Literal(null, StringType)) as 'c14,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
index 052b0a9196..cc575bedd8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala
@@ -205,9 +205,9 @@ object ParquetFilters {
Some(new AndFilter(leftFilter.get, rightFilter.get))
}
}
- case p @ Equals(left: Literal, right: NamedExpression) if !right.nullable =>
+ case p @ EqualTo(left: Literal, right: NamedExpression) if !right.nullable =>
Some(createEqualityFilter(right.name, left, p))
- case p @ Equals(left: NamedExpression, right: Literal) if !left.nullable =>
+ case p @ EqualTo(left: NamedExpression, right: Literal) if !left.nullable =>
Some(createEqualityFilter(left.name, right, p))
case p @ LessThan(left: Literal, right: NamedExpression) if !right.nullable =>
Some(createLessThanFilter(right.name, left, p))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
index 0c239d00b1..7714eb1b56 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala
@@ -19,27 +19,23 @@ package org.apache.spark.sql.parquet
import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
-import org.apache.hadoop.fs.{Path, FileSystem}
-import org.apache.hadoop.mapreduce.Job
-
import parquet.hadoop.ParquetFileWriter
import parquet.hadoop.util.ContextUtil
import parquet.schema.MessageTypeParser
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.mapreduce.Job
import org.apache.spark.SparkContext
import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser}
+import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types.{BooleanType, IntegerType}
-import org.apache.spark.sql.test.TestSQLContext
-import org.apache.spark.sql.TestData
-import org.apache.spark.sql.SchemaRDD
import org.apache.spark.sql.catalyst.util.getTempFilePath
-import org.apache.spark.sql.catalyst.{SqlLexical, SqlParser}
-import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, Star}
+import org.apache.spark.sql.test.TestSQLContext
+import org.apache.spark.sql.test.TestSQLContext._
import org.apache.spark.util.Utils
-// Implicits
-import org.apache.spark.sql.test.TestSQLContext._
case class TestRDDEntry(key: Int, value: String)
@@ -72,7 +68,6 @@ case class AllDataTypes(
booleanField: Boolean)
class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll {
- import TestData._
TestData // Load test data tables.
var testRDD: SchemaRDD = null
@@ -319,7 +314,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA
test("create RecordFilter for simple predicates") {
val attribute1 = new AttributeReference("first", IntegerType, false)()
- val predicate1 = new Equals(attribute1, new Literal(1, IntegerType))
+ val predicate1 = new EqualTo(attribute1, new Literal(1, IntegerType))
val filter1 = ParquetFilters.createFilter(predicate1)
assert(filter1.isDefined)
assert(filter1.get.predicate == predicate1, "predicates do not match")
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
}