Repository: spark
Updated Branches:
  refs/heads/master 3a342dedc -> 9fd13d561


[SPARK-8770][SQL] Create BinaryOperator abstract class.

Our current BinaryExpression abstract class is not for generic binary 
expressions, i.e. it requires left/right children to have the same type. 
However, due to its name, contributors build new binary expressions that don't 
have that assumption (e.g. Sha) and still extend BinaryExpression.

This patch creates a new BinaryOperator abstract class, and update the analyzer 
o only apply type casting rule there. This patch also adds the notion of 
"prettyName" to expressions, which defines the user-facing name for the 
expression.

Author: Reynold Xin <r...@databricks.com>

Closes #7174 from rxin/binary-opterator and squashes the following commits:

f31900d [Reynold Xin] [SPARK-8770][SQL] Create BinaryOperator abstract class.
fceb216 [Reynold Xin] Merge branch 'master' of github.com:apache/spark into 
binary-opterator
d8518cf [Reynold Xin] Updated Python tests.


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

Branch: refs/heads/master
Commit: 9fd13d5613b6d16a78d97d4798f085b56107d343
Parents: 3a342de
Author: Reynold Xin <r...@databricks.com>
Authored: Wed Jul 1 21:14:13 2015 -0700
Committer: Reynold Xin <r...@databricks.com>
Committed: Wed Jul 1 21:14:13 2015 -0700

----------------------------------------------------------------------
 python/pyspark/sql/dataframe.py                 |  10 +-
 python/pyspark/sql/functions.py                 |   4 +-
 python/pyspark/sql/group.py                     |  24 +--
 .../catalyst/analysis/HiveTypeCoercion.scala    |  17 +-
 .../expressions/ExpectsInputTypes.scala         |  59 +++++++
 .../sql/catalyst/expressions/Expression.scala   | 161 +++++++++----------
 .../sql/catalyst/expressions/ScalaUDF.scala     |   2 +-
 .../sql/catalyst/expressions/aggregates.scala   |   9 +-
 .../sql/catalyst/expressions/arithmetic.scala   |  14 +-
 .../expressions/complexTypeCreator.scala        |   4 +-
 .../catalyst/expressions/nullFunctions.scala    |   2 -
 .../sql/catalyst/expressions/predicates.scala   |   6 +-
 .../spark/sql/catalyst/expressions/sets.scala   |   2 -
 .../catalyst/expressions/stringOperations.scala |  26 +--
 .../sql/catalyst/trees/TreeNodeSuite.scala      |   6 +-
 15 files changed, 191 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/python/pyspark/sql/dataframe.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py
index 273a40d..1e9c657 100644
--- a/python/pyspark/sql/dataframe.py
+++ b/python/pyspark/sql/dataframe.py
@@ -802,11 +802,11 @@ class DataFrame(object):
             Each element should be a column name (string) or an expression 
(:class:`Column`).
 
         >>> df.groupBy().avg().collect()
-        [Row(AVG(age)=3.5)]
+        [Row(avg(age)=3.5)]
         >>> df.groupBy('name').agg({'age': 'mean'}).collect()
-        [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)]
+        [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)]
         >>> df.groupBy(df.name).avg().collect()
-        [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)]
+        [Row(name=u'Alice', avg(age)=2.0), Row(name=u'Bob', avg(age)=5.0)]
         >>> df.groupBy(['name', df.age]).count().collect()
         [Row(name=u'Bob', age=5, count=1), Row(name=u'Alice', age=2, count=1)]
         """
@@ -864,10 +864,10 @@ class DataFrame(object):
         (shorthand for ``df.groupBy.agg()``).
 
         >>> df.agg({"age": "max"}).collect()
-        [Row(MAX(age)=5)]
+        [Row(max(age)=5)]
         >>> from pyspark.sql import functions as F
         >>> df.agg(F.min(df.age)).collect()
-        [Row(MIN(age)=2)]
+        [Row(min(age)=2)]
         """
         return self.groupBy().agg(*exprs)
 

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/python/pyspark/sql/functions.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index 4e2be88..f9a15d4 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -266,7 +266,7 @@ def coalesce(*cols):
 
     >>> cDf.select(coalesce(cDf["a"], cDf["b"])).show()
     +-------------+
-    |Coalesce(a,b)|
+    |coalesce(a,b)|
     +-------------+
     |         null|
     |            1|
@@ -275,7 +275,7 @@ def coalesce(*cols):
 
     >>> cDf.select('*', coalesce(cDf["a"], lit(0.0))).show()
     +----+----+---------------+
-    |   a|   b|Coalesce(a,0.0)|
+    |   a|   b|coalesce(a,0.0)|
     +----+----+---------------+
     |null|null|            0.0|
     |   1|null|            1.0|

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/python/pyspark/sql/group.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/group.py b/python/pyspark/sql/group.py
index 5a37a67..04594d5 100644
--- a/python/pyspark/sql/group.py
+++ b/python/pyspark/sql/group.py
@@ -75,11 +75,11 @@ class GroupedData(object):
 
         >>> gdf = df.groupBy(df.name)
         >>> gdf.agg({"*": "count"}).collect()
-        [Row(name=u'Alice', COUNT(1)=1), Row(name=u'Bob', COUNT(1)=1)]
+        [Row(name=u'Alice', count(1)=1), Row(name=u'Bob', count(1)=1)]
 
         >>> from pyspark.sql import functions as F
         >>> gdf.agg(F.min(df.age)).collect()
-        [Row(name=u'Alice', MIN(age)=2), Row(name=u'Bob', MIN(age)=5)]
+        [Row(name=u'Alice', min(age)=2), Row(name=u'Bob', min(age)=5)]
         """
         assert exprs, "exprs should not be empty"
         if len(exprs) == 1 and isinstance(exprs[0], dict):
@@ -110,9 +110,9 @@ class GroupedData(object):
         :param cols: list of column names (string). Non-numeric columns are 
ignored.
 
         >>> df.groupBy().mean('age').collect()
-        [Row(AVG(age)=3.5)]
+        [Row(avg(age)=3.5)]
         >>> df3.groupBy().mean('age', 'height').collect()
-        [Row(AVG(age)=3.5, AVG(height)=82.5)]
+        [Row(avg(age)=3.5, avg(height)=82.5)]
         """
 
     @df_varargs_api
@@ -125,9 +125,9 @@ class GroupedData(object):
         :param cols: list of column names (string). Non-numeric columns are 
ignored.
 
         >>> df.groupBy().avg('age').collect()
-        [Row(AVG(age)=3.5)]
+        [Row(avg(age)=3.5)]
         >>> df3.groupBy().avg('age', 'height').collect()
-        [Row(AVG(age)=3.5, AVG(height)=82.5)]
+        [Row(avg(age)=3.5, avg(height)=82.5)]
         """
 
     @df_varargs_api
@@ -136,9 +136,9 @@ class GroupedData(object):
         """Computes the max value for each numeric columns for each group.
 
         >>> df.groupBy().max('age').collect()
-        [Row(MAX(age)=5)]
+        [Row(max(age)=5)]
         >>> df3.groupBy().max('age', 'height').collect()
-        [Row(MAX(age)=5, MAX(height)=85)]
+        [Row(max(age)=5, max(height)=85)]
         """
 
     @df_varargs_api
@@ -149,9 +149,9 @@ class GroupedData(object):
         :param cols: list of column names (string). Non-numeric columns are 
ignored.
 
         >>> df.groupBy().min('age').collect()
-        [Row(MIN(age)=2)]
+        [Row(min(age)=2)]
         >>> df3.groupBy().min('age', 'height').collect()
-        [Row(MIN(age)=2, MIN(height)=80)]
+        [Row(min(age)=2, min(height)=80)]
         """
 
     @df_varargs_api
@@ -162,9 +162,9 @@ class GroupedData(object):
         :param cols: list of column names (string). Non-numeric columns are 
ignored.
 
         >>> df.groupBy().sum('age').collect()
-        [Row(SUM(age)=7)]
+        [Row(sum(age)=7)]
         >>> df3.groupBy().sum('age', 'height').collect()
-        [Row(SUM(age)=7, SUM(height)=165)]
+        [Row(sum(age)=7, sum(height)=165)]
         """
 
 

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/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 2ab5cb6..8420c54 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
@@ -150,6 +150,7 @@ object HiveTypeCoercion {
    * Converts string "NaN"s that are in binary operators with a NaN-able types 
(Float / Double) to
    * the appropriate numeric equivalent.
    */
+  // TODO: remove this rule and make Cast handle Nan.
   object ConvertNaNs extends Rule[LogicalPlan] {
     private val StringNaN = Literal("NaN")
 
@@ -159,19 +160,19 @@ object HiveTypeCoercion {
         case e if !e.childrenResolved => e
 
         /* Double Conversions */
-        case b @ BinaryExpression(StringNaN, right @ DoubleType()) =>
+        case b @ BinaryOperator(StringNaN, right @ DoubleType()) =>
           b.makeCopy(Array(Literal(Double.NaN), right))
-        case b @ BinaryExpression(left @ DoubleType(), StringNaN) =>
+        case b @ BinaryOperator(left @ DoubleType(), StringNaN) =>
           b.makeCopy(Array(left, Literal(Double.NaN)))
 
         /* Float Conversions */
-        case b @ BinaryExpression(StringNaN, right @ FloatType()) =>
+        case b @ BinaryOperator(StringNaN, right @ FloatType()) =>
           b.makeCopy(Array(Literal(Float.NaN), right))
-        case b @ BinaryExpression(left @ FloatType(), StringNaN) =>
+        case b @ BinaryOperator(left @ FloatType(), StringNaN) =>
           b.makeCopy(Array(left, Literal(Float.NaN)))
 
         /* Use float NaN by default to avoid unnecessary type widening */
-        case b @ BinaryExpression(left @ StringNaN, StringNaN) =>
+        case b @ BinaryOperator(left @ StringNaN, StringNaN) =>
           b.makeCopy(Array(left, Literal(Float.NaN)))
       }
     }
@@ -245,12 +246,12 @@ object HiveTypeCoercion {
 
         Union(newLeft, newRight)
 
-      // Also widen types for BinaryExpressions.
+      // Also widen types for BinaryOperator.
       case q: LogicalPlan => q transformExpressions {
         // Skip nodes who's children have not been resolved yet.
         case e if !e.childrenResolved => e
 
-        case b @ BinaryExpression(left, right) if left.dataType != 
right.dataType =>
+        case b @ BinaryOperator(left, right) if left.dataType != 
right.dataType =>
           findTightestCommonTypeOfTwo(left.dataType, right.dataType).map { 
widestType =>
             val newLeft = if (left.dataType == widestType) left else 
Cast(left, widestType)
             val newRight = if (right.dataType == widestType) right else 
Cast(right, widestType)
@@ -478,7 +479,7 @@ object HiveTypeCoercion {
 
         // Promote integers inside a binary expression with fixed-precision 
decimals to decimals,
         // and fixed-precision decimals in an expression with floats / doubles 
to doubles
-        case b @ BinaryExpression(left, right) if left.dataType != 
right.dataType =>
+        case b @ BinaryOperator(left, right) if left.dataType != 
right.dataType =>
           (left.dataType, right.dataType) match {
             case (t, DecimalType.Fixed(p, s)) if intTypeToFixed.contains(t) =>
               b.makeCopy(Array(Cast(left, intTypeToFixed(t)), right))

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala
new file mode 100644
index 0000000..450fc41
--- /dev/null
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExpectsInputTypes.scala
@@ -0,0 +1,59 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.expressions
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.types.DataType
+
+
+/**
+ * An trait that gets mixin to define the expected input types of an 
expression.
+ */
+trait ExpectsInputTypes { self: Expression =>
+
+  /**
+   * Expected input types from child expressions. The i-th position in the 
returned seq indicates
+   * the type requirement for the i-th child.
+   *
+   * The possible values at each position are:
+   * 1. a specific data type, e.g. LongType, StringType.
+   * 2. a non-leaf data type, e.g. NumericType, IntegralType, FractionalType.
+   * 3. a list of specific data types, e.g. Seq(StringType, BinaryType).
+   */
+  def inputTypes: Seq[Any]
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    // We will do the type checking in `HiveTypeCoercion`, so always returning 
success here.
+    TypeCheckResult.TypeCheckSuccess
+  }
+}
+
+/**
+ * Expressions that require a specific `DataType` as input should implement 
this trait
+ * so that the proper type conversions can be performed in the analyzer.
+ */
+trait AutoCastInputTypes { self: Expression =>
+
+  def inputTypes: Seq[DataType]
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    // We will always do type casting for `AutoCastInputTypes` in 
`HiveTypeCoercion`,
+    // so type mismatch error won't be reported here, but for underling 
`Cast`s.
+    TypeCheckResult.TypeCheckSuccess
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index e18a311..cafbbaf 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -120,17 +120,6 @@ abstract class Expression extends TreeNode[Expression] {
   def childrenResolved: Boolean = children.forall(_.resolved)
 
   /**
-   * Returns a string representation of this expression that does not have 
developer centric
-   * debugging information like the expression id.
-   */
-  def prettyString: String = {
-    transform {
-      case a: AttributeReference => PrettyAttribute(a.name)
-      case u: UnresolvedAttribute => PrettyAttribute(u.name)
-    }.toString
-  }
-
-  /**
    * Returns true when two expressions will always compute the same result, 
even if they differ
    * cosmetically (i.e. capitalization of names in attributes may be 
different).
    */
@@ -154,71 +143,40 @@ abstract class Expression extends TreeNode[Expression] {
    * Note: it's not valid to call this method until `childrenResolved == true`.
    */
   def checkInputDataTypes(): TypeCheckResult = TypeCheckResult.TypeCheckSuccess
-}
-
-abstract class BinaryExpression extends Expression with 
trees.BinaryNode[Expression] {
-  self: Product =>
-
-  def symbol: String = sys.error(s"BinaryExpressions must override either 
toString or symbol")
-
-  override def foldable: Boolean = left.foldable && right.foldable
-
-  override def nullable: Boolean = left.nullable || right.nullable
-
-  override def toString: String = s"($left $symbol $right)"
 
   /**
-   * Short hand for generating binary evaluation code.
-   * If either of the sub-expressions is null, the result of this computation
-   * is assumed to be null.
-   *
-   * @param f accepts two variable names and returns Java code to compute the 
output.
+   * Returns a user-facing string representation of this expression's name.
+   * This should usually match the name of the function in SQL.
    */
-  protected def defineCodeGen(
-      ctx: CodeGenContext,
-      ev: GeneratedExpressionCode,
-      f: (String, String) => String): String = {
-    nullSafeCodeGen(ctx, ev, (result, eval1, eval2) => {
-      s"$result = ${f(eval1, eval2)};"
-    })
-  }
+  def prettyName: String = getClass.getSimpleName.toLowerCase
 
   /**
-   * Short hand for generating binary evaluation code.
-   * If either of the sub-expressions is null, the result of this computation
-   * is assumed to be null.
+   * Returns a user-facing string representation of this expression, i.e. does 
not have developer
+   * centric debugging information like the expression id.
    */
-  protected def nullSafeCodeGen(
-      ctx: CodeGenContext,
-      ev: GeneratedExpressionCode,
-      f: (String, String, String) => String): String = {
-    val eval1 = left.gen(ctx)
-    val eval2 = right.gen(ctx)
-    val resultCode = f(ev.primitive, eval1.primitive, eval2.primitive)
-    s"""
-      ${eval1.code}
-      boolean ${ev.isNull} = ${eval1.isNull};
-      ${ctx.javaType(dataType)} ${ev.primitive} = 
${ctx.defaultValue(dataType)};
-      if (!${ev.isNull}) {
-        ${eval2.code}
-        if (!${eval2.isNull}) {
-          $resultCode
-        } else {
-          ${ev.isNull} = true;
-        }
-      }
-    """
+  def prettyString: String = {
+    transform {
+      case a: AttributeReference => PrettyAttribute(a.name)
+      case u: UnresolvedAttribute => PrettyAttribute(u.name)
+    }.toString
   }
-}
 
-private[sql] object BinaryExpression {
-  def unapply(e: BinaryExpression): Option[(Expression, Expression)] = 
Some((e.left, e.right))
+  override def toString: String = prettyName + children.mkString("(", ",", ")")
 }
 
+
+/**
+ * A leaf expression, i.e. one without any child expressions.
+ */
 abstract class LeafExpression extends Expression with 
trees.LeafNode[Expression] {
   self: Product =>
 }
 
+
+/**
+ * An expression with one input and one output. The output is by default 
evaluated to null
+ * if the input is evaluated to null.
+ */
 abstract class UnaryExpression extends Expression with 
trees.UnaryNode[Expression] {
   self: Product =>
 
@@ -265,39 +223,76 @@ abstract class UnaryExpression extends Expression with 
trees.UnaryNode[Expressio
   }
 }
 
+
 /**
- * An trait that gets mixin to define the expected input types of an 
expression.
+ * An expression with two inputs and one output. The output is by default 
evaluated to null
+ * if any input is evaluated to null.
  */
-trait ExpectsInputTypes { self: Expression =>
+abstract class BinaryExpression extends Expression with 
trees.BinaryNode[Expression] {
+  self: Product =>
+
+  override def foldable: Boolean = left.foldable && right.foldable
+
+  override def nullable: Boolean = left.nullable || right.nullable
 
   /**
-   * Expected input types from child expressions. The i-th position in the 
returned seq indicates
-   * the type requirement for the i-th child.
+   * Short hand for generating binary evaluation code.
+   * If either of the sub-expressions is null, the result of this computation
+   * is assumed to be null.
    *
-   * The possible values at each position are:
-   * 1. a specific data type, e.g. LongType, StringType.
-   * 2. a non-leaf data type, e.g. NumericType, IntegralType, FractionalType.
-   * 3. a list of specific data types, e.g. Seq(StringType, BinaryType).
+   * @param f accepts two variable names and returns Java code to compute the 
output.
    */
-  def inputTypes: Seq[Any]
+  protected def defineCodeGen(
+    ctx: CodeGenContext,
+    ev: GeneratedExpressionCode,
+    f: (String, String) => String): String = {
+    nullSafeCodeGen(ctx, ev, (result, eval1, eval2) => {
+      s"$result = ${f(eval1, eval2)};"
+    })
+  }
 
-  override def checkInputDataTypes(): TypeCheckResult = {
-    // We will do the type checking in `HiveTypeCoercion`, so always returning 
success here.
-    TypeCheckResult.TypeCheckSuccess
+  /**
+   * Short hand for generating binary evaluation code.
+   * If either of the sub-expressions is null, the result of this computation
+   * is assumed to be null.
+   */
+  protected def nullSafeCodeGen(
+    ctx: CodeGenContext,
+    ev: GeneratedExpressionCode,
+    f: (String, String, String) => String): String = {
+    val eval1 = left.gen(ctx)
+    val eval2 = right.gen(ctx)
+    val resultCode = f(ev.primitive, eval1.primitive, eval2.primitive)
+    s"""
+      ${eval1.code}
+      boolean ${ev.isNull} = ${eval1.isNull};
+      ${ctx.javaType(dataType)} ${ev.primitive} = 
${ctx.defaultValue(dataType)};
+      if (!${ev.isNull}) {
+        ${eval2.code}
+        if (!${eval2.isNull}) {
+          $resultCode
+        } else {
+          ${ev.isNull} = true;
+        }
+      }
+    """
   }
 }
 
+
 /**
- * Expressions that require a specific `DataType` as input should implement 
this trait
- * so that the proper type conversions can be performed in the analyzer.
+ * An expression that has two inputs that are expected to the be same type. If 
the two inputs have
+ * different types, the analyzer will find the tightest common type and do the 
proper type casting.
  */
-trait AutoCastInputTypes { self: Expression =>
+abstract class BinaryOperator extends BinaryExpression {
+  self: Product =>
 
-  def inputTypes: Seq[DataType]
+  def symbol: String
 
-  override def checkInputDataTypes(): TypeCheckResult = {
-    // We will always do type casting for `AutoCastInputTypes` in 
`HiveTypeCoercion`,
-    // so type mismatch error won't be reported here, but for underling 
`Cast`s.
-    TypeCheckResult.TypeCheckSuccess
-  }
+  override def toString: String = s"($left $symbol $right)"
+}
+
+
+private[sql] object BinaryOperator {
+  def unapply(e: BinaryOperator): Option[(Expression, Expression)] = 
Some((e.left, e.right))
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala
index ebabb6f..caf021b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala
@@ -29,7 +29,7 @@ case class ScalaUDF(function: AnyRef, dataType: DataType, 
children: Seq[Expressi
 
   override def nullable: Boolean = true
 
-  override def toString: String = s"scalaUDF(${children.mkString(",")})"
+  override def toString: String = s"UDF(${children.mkString(",")})"
 
   // scalastyle:off
 

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index a9fc54c..64e07bd 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -94,7 +94,6 @@ case class Min(child: Expression) extends PartialAggregate 
with trees.UnaryNode[
 
   override def nullable: Boolean = true
   override def dataType: DataType = child.dataType
-  override def toString: String = s"MIN($child)"
 
   override def asPartial: SplitEvaluation = {
     val partialMin = Alias(Min(child), "PartialMin")()
@@ -128,7 +127,6 @@ case class Max(child: Expression) extends PartialAggregate 
with trees.UnaryNode[
 
   override def nullable: Boolean = true
   override def dataType: DataType = child.dataType
-  override def toString: String = s"MAX($child)"
 
   override def asPartial: SplitEvaluation = {
     val partialMax = Alias(Max(child), "PartialMax")()
@@ -162,7 +160,6 @@ case class Count(child: Expression) extends 
PartialAggregate with trees.UnaryNod
 
   override def nullable: Boolean = false
   override def dataType: LongType.type = LongType
-  override def toString: String = s"COUNT($child)"
 
   override def asPartial: SplitEvaluation = {
     val partialCount = Alias(Count(child), "PartialCount")()
@@ -390,6 +387,8 @@ case class ApproxCountDistinct(child: Expression, 
relativeSD: Double = 0.05)
 
 case class Average(child: Expression) extends PartialAggregate with 
trees.UnaryNode[Expression] {
 
+  override def prettyName: String = "avg"
+
   override def nullable: Boolean = true
 
   override def dataType: DataType = child.dataType match {
@@ -401,8 +400,6 @@ case class Average(child: Expression) extends 
PartialAggregate with trees.UnaryN
       DoubleType
   }
 
-  override def toString: String = s"AVG($child)"
-
   override def asPartial: SplitEvaluation = {
     child.dataType match {
       case DecimalType.Fixed(_, _) | DecimalType.Unlimited =>
@@ -494,8 +491,6 @@ case class Sum(child: Expression) extends PartialAggregate 
with trees.UnaryNode[
       child.dataType
   }
 
-  override def toString: String = s"SUM($child)"
-
   override def asPartial: SplitEvaluation = {
     child.dataType match {
       case DecimalType.Fixed(_, _) =>

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 5363b35..4fbf4c8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -57,7 +57,7 @@ case class UnaryMinus(child: Expression) extends 
UnaryArithmetic {
 }
 
 case class UnaryPositive(child: Expression) extends UnaryArithmetic {
-  override def toString: String = s"positive($child)"
+  override def prettyName: String = "positive"
 
   override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): 
String =
     defineCodeGen(ctx, ev, c => c)
@@ -69,8 +69,6 @@ case class UnaryPositive(child: Expression) extends 
UnaryArithmetic {
  * A function that get the absolute value of the numeric value.
  */
 case class Abs(child: Expression) extends UnaryArithmetic {
-  override def toString: String = s"Abs($child)"
-
   override def checkInputDataTypes(): TypeCheckResult =
     TypeUtils.checkForNumericExpr(child.dataType, "function abs")
 
@@ -79,10 +77,9 @@ case class Abs(child: Expression) extends UnaryArithmetic {
   protected override def evalInternal(evalE: Any) = numeric.abs(evalE)
 }
 
-abstract class BinaryArithmetic extends BinaryExpression {
+abstract class BinaryArithmetic extends BinaryOperator {
   self: Product =>
 
-
   override def dataType: DataType = left.dataType
 
   override def checkInputDataTypes(): TypeCheckResult = {
@@ -360,7 +357,9 @@ case class MaxOf(left: Expression, right: Expression) 
extends BinaryArithmetic {
       }
     """
   }
-  override def toString: String = s"MaxOf($left, $right)"
+
+  override def symbol: String = "max"
+  override def prettyName: String = symbol
 }
 
 case class MinOf(left: Expression, right: Expression) extends BinaryArithmetic 
{
@@ -413,5 +412,6 @@ case class MinOf(left: Expression, right: Expression) 
extends BinaryArithmetic {
     """
   }
 
-  override def toString: String = s"MinOf($left, $right)"
+  override def symbol: String = "min"
+  override def prettyName: String = symbol
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
index 5def57b..67e7dc4 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala
@@ -43,7 +43,7 @@ case class CreateArray(children: Seq[Expression]) extends 
Expression {
     children.map(_.eval(input))
   }
 
-  override def toString: String = s"Array(${children.mkString(",")})"
+  override def prettyName: String = "array"
 }
 
 /**
@@ -71,4 +71,6 @@ case class CreateStruct(children: Seq[Expression]) extends 
Expression {
   override def eval(input: InternalRow): Any = {
     InternalRow(children.map(_.eval(input)): _*)
   }
+
+  override def prettyName: String = "struct"
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
index 78be282..145d323 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
@@ -38,8 +38,6 @@ case class Coalesce(children: Seq[Expression]) extends 
Expression {
     }
   }
 
-  override def toString: String = s"Coalesce(${children.mkString(",")})"
-
   override def dataType: DataType = children.head.dataType
 
   override def eval(input: InternalRow): Any = {

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
----------------------------------------------------------------------
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 a777f77..34df89a 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
@@ -120,7 +120,7 @@ case class InSet(value: Expression, hset: Set[Any])
 }
 
 case class And(left: Expression, right: Expression)
-  extends BinaryExpression with Predicate with AutoCastInputTypes {
+  extends BinaryOperator with Predicate with AutoCastInputTypes {
 
   override def inputTypes: Seq[DataType] = Seq(BooleanType, BooleanType)
 
@@ -169,7 +169,7 @@ case class And(left: Expression, right: Expression)
 }
 
 case class Or(left: Expression, right: Expression)
-  extends BinaryExpression with Predicate with AutoCastInputTypes {
+  extends BinaryOperator with Predicate with AutoCastInputTypes {
 
   override def inputTypes: Seq[DataType] = Seq(BooleanType, BooleanType)
 
@@ -217,7 +217,7 @@ case class Or(left: Expression, right: Expression)
   }
 }
 
-abstract class BinaryComparison extends BinaryExpression with Predicate {
+abstract class BinaryComparison extends BinaryOperator with Predicate {
   self: Product =>
 
   override def checkInputDataTypes(): TypeCheckResult = {

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
index daa9f44..5d51a4c 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala
@@ -137,8 +137,6 @@ case class CombineSets(left: Expression, right: Expression) 
extends BinaryExpres
 
   override def dataType: DataType = left.dataType
 
-  override def symbol: String = "++="
-
   override def eval(input: InternalRow): Any = {
     val leftEval = left.eval(input).asInstanceOf[OpenHashSet[Any]]
     if(leftEval != null) {

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index 4cbfc4e..b020f2b 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -75,8 +75,6 @@ trait StringRegexExpression extends AutoCastInputTypes {
 case class Like(left: Expression, right: Expression)
   extends BinaryExpression with StringRegexExpression {
 
-  override def symbol: String = "LIKE"
-
   // replace the _ with .{1} exactly match 1 time of any character
   // replace the % with .*, match 0 or more times with any character
   override def escape(v: String): String =
@@ -101,14 +99,16 @@ case class Like(left: Expression, right: Expression)
     }
 
   override def matches(regex: Pattern, str: String): Boolean = 
regex.matcher(str).matches()
+
+  override def toString: String = s"$left LIKE $right"
 }
 
 case class RLike(left: Expression, right: Expression)
   extends BinaryExpression with StringRegexExpression {
 
-  override def symbol: String = "RLIKE"
   override def escape(v: String): String = v
   override def matches(regex: Pattern, str: String): Boolean = 
regex.matcher(str).find(0)
+  override def toString: String = s"$left RLIKE $right"
 }
 
 trait CaseConversionExpression extends AutoCastInputTypes {
@@ -134,9 +134,7 @@ trait CaseConversionExpression extends AutoCastInputTypes {
  */
 case class Upper(child: Expression) extends UnaryExpression with 
CaseConversionExpression {
 
-  override def convert(v: UTF8String): UTF8String = v.toUpperCase()
-
-  override def toString: String = s"Upper($child)"
+  override def convert(v: UTF8String): UTF8String = v.toUpperCase
 
   override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): 
String = {
     defineCodeGen(ctx, ev, c => s"($c).toUpperCase()")
@@ -148,9 +146,7 @@ case class Upper(child: Expression) extends UnaryExpression 
with CaseConversionE
  */
 case class Lower(child: Expression) extends UnaryExpression with 
CaseConversionExpression {
 
-  override def convert(v: UTF8String): UTF8String = v.toLowerCase()
-
-  override def toString: String = s"Lower($child)"
+  override def convert(v: UTF8String): UTF8String = v.toLowerCase
 
   override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): 
String = {
     defineCodeGen(ctx, ev, c => s"($c).toLowerCase()")
@@ -178,8 +174,6 @@ trait StringComparison extends AutoCastInputTypes {
     }
   }
 
-  override def symbol: String = nodeName
-
   override def toString: String = s"$nodeName($left, $right)"
 }
 
@@ -284,12 +278,6 @@ case class Substring(str: Expression, pos: Expression, 
len: Expression)
       }
     }
   }
-
-  override def toString: String = len match {
-    // TODO: This is broken because max is not an integer value.
-    case max if max == Integer.MAX_VALUE => s"SUBSTR($str, $pos)"
-    case _ => s"SUBSTR($str, $pos, $len)"
-  }
 }
 
 /**
@@ -304,9 +292,9 @@ case class StringLength(child: Expression) extends 
UnaryExpression with AutoCast
     if (string == null) null else string.asInstanceOf[UTF8String].length
   }
 
-  override def toString: String = s"length($child)"
-
   override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): 
String = {
     defineCodeGen(ctx, ev, c => s"($c).length()")
   }
+
+  override def prettyName: String = "length"
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/9fd13d56/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
----------------------------------------------------------------------
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
index bda2179..86792f0 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala
@@ -73,7 +73,7 @@ class TreeNodeSuite extends SparkFunSuite {
     val expected = Seq("+", "1", "*", "2", "-", "3", "4")
     val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), 
Literal(4))))
     expression transformDown {
-      case b: BinaryExpression => actual.append(b.symbol); b
+      case b: BinaryOperator => actual.append(b.symbol); b
       case l: Literal => actual.append(l.toString); l
     }
 
@@ -85,7 +85,7 @@ class TreeNodeSuite extends SparkFunSuite {
     val expected = Seq("1", "2", "3", "4", "-", "*", "+")
     val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), 
Literal(4))))
     expression transformUp {
-      case b: BinaryExpression => actual.append(b.symbol); b
+      case b: BinaryOperator => actual.append(b.symbol); b
       case l: Literal => actual.append(l.toString); l
     }
 
@@ -125,7 +125,7 @@ class TreeNodeSuite extends SparkFunSuite {
     val expected = Seq("1", "2", "3", "4", "-", "*", "+")
     val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), 
Literal(4))))
     expression foreachUp {
-      case b: BinaryExpression => actual.append(b.symbol);
+      case b: BinaryOperator => actual.append(b.symbol);
       case l: Literal => actual.append(l.toString);
     }
 


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org
For additional commands, e-mail: commits-h...@spark.apache.org

Reply via email to