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 46fcfbb9e26ba..3eb93a37b7af8 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 @@ -309,13 +309,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 => EqualsTo(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(EqualsTo(e1, e2)) } | + termExpression ~ "<>" ~ termExpression ^^ { case e1 ~ _ ~ e2 => Not(EqualsTo(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 c0714bcdd0afb..c2d7e0d30725a 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 EqualsTo operators as that actually makes sense for boolean types. + case e: EqualsTo => 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,7 @@ 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))) + case Cast(e, BooleanType) => Not(EqualsTo(e, Literal(0))) 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 d177339d40ae5..64be4294e76ab 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.EqualsTo = ('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) = EqualsTo(expr, other) + def !== (other: Expression) = Not(EqualsTo(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 573ec052f4266..4f301734ecb1e 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]], [[EqualsTo]]), 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 2902906df2844..8cfaf93b6b556 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(EqualsTo(a,b), R)` returns `true` where as `canEvaluate(EqualsTo(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 EqualsTo(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 820ecfb78b52e..c61f60c2cd559 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 EqualsTo(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 EqualsTo(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => (l, r) + case EqualsTo(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 cea97c584f7e1..81fbf544facf5 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, + EqualsTo(Literal(null, IntegerType), 1) as 'c11, + EqualsTo(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 052b0a9196717..a4d8c9ae8d99e 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 @ EqualsTo(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 @ EqualsTo(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 9810520bb9ae6..e0204eeb2cb79 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 @@ -33,7 +33,7 @@ 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.expressions.Row -import org.apache.spark.sql.catalyst.expressions.Equals +import org.apache.spark.sql.catalyst.expressions.EqualsTo import org.apache.spark.sql.catalyst.types.IntegerType import org.apache.spark.util.Utils @@ -245,7 +245,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 EqualsTo(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 844673f66d103..11b2a3ebf262e 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 @@ -656,7 +656,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) => EqualsTo(e1, e2): Expression } predicates.reduceLeft(And) }.toBuffer @@ -886,9 +886,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) => EqualsTo(nodeToExpr(left), nodeToExpr(right)) + case Token("!=", left :: right:: Nil) => Not(EqualsTo(nodeToExpr(left), nodeToExpr(right))) + case Token("<>", left :: right:: Nil) => Not(EqualsTo(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)) @@ -928,7 +928,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(EqualsTo(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 cc8744c9668eb..fbb8d2e2345eb 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, EqualsTo} import org.apache.spark.sql.execution.Project import org.apache.spark.sql.hive.test.TestHive @@ -43,9 +43,9 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { c } - // Only one Equals + // Only one EqualsTo var numEquals = 0 - project.transformAllExpressions { case e: Equals => + project.transformAllExpressions { case e: EqualsTo => numEquals += 1 e }