diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 5f76a512679a434a5488cf5af183088ae3922ac5..2a1f96409daf4bc9f195aa1099e62f0d8eb053dd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -161,7 +161,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w
         try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null }
       })
     case BooleanType =>
-      buildCast[Boolean](_, b => new Timestamp((if (b) 1 else 0)))
+      buildCast[Boolean](_, b => new Timestamp(if (b) 1 else 0))
     case LongType =>
       buildCast[Long](_, l => new Timestamp(l))
     case IntegerType =>
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 0ed576b3d5870bdffbd31b85737ae8251632af5d..432d65eee54fb5e5a2ca8e1e147e7bd2cd372186 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
@@ -81,11 +81,11 @@ abstract class Expression extends TreeNode[Expression] {
     val objectTerm = ctx.freshName("obj")
     s"""
       /* expression: ${this} */
-      Object ${objectTerm} = expressions[${ctx.references.size - 1}].eval(i);
-      boolean ${ev.isNull} = ${objectTerm} == null;
+      Object $objectTerm = expressions[${ctx.references.size - 1}].eval(i);
+      boolean ${ev.isNull} = $objectTerm == null;
       ${ctx.javaType(this.dataType)} ${ev.primitive} = ${ctx.defaultValue(this.dataType)};
       if (!${ev.isNull}) {
-        ${ev.primitive} = (${ctx.boxedType(this.dataType)})${objectTerm};
+        ${ev.primitive} = (${ctx.boxedType(this.dataType)}) $objectTerm;
       }
     """
   }
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 3ac7c92dcd00996336833830dc4d43175e04c8ba..d4efda2e04c2994c8ee0a8b79682729661380f05 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
@@ -322,102 +322,6 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet
   }
 }
 
-/**
- * A function that calculates bitwise and(&) of two numbers.
- */
-case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def symbol: String = "&"
-
-  protected def checkTypesInternal(t: DataType) =
-    TypeUtils.checkForBitwiseExpr(t, "operator " + symbol)
-
-  private lazy val and: (Any, Any) => Any = dataType match {
-    case ByteType =>
-      ((evalE1: Byte, evalE2: Byte) => (evalE1 & evalE2).toByte).asInstanceOf[(Any, Any) => Any]
-    case ShortType =>
-      ((evalE1: Short, evalE2: Short) => (evalE1 & evalE2).toShort).asInstanceOf[(Any, Any) => Any]
-    case IntegerType =>
-      ((evalE1: Int, evalE2: Int) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any]
-    case LongType =>
-      ((evalE1: Long, evalE2: Long) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any]
-  }
-
-  protected override def evalInternal(evalE1: Any, evalE2: Any) = and(evalE1, evalE2)
-}
-
-/**
- * A function that calculates bitwise or(|) of two numbers.
- */
-case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def symbol: String = "|"
-
-  protected def checkTypesInternal(t: DataType) =
-    TypeUtils.checkForBitwiseExpr(t, "operator " + symbol)
-
-  private lazy val or: (Any, Any) => Any = dataType match {
-    case ByteType =>
-      ((evalE1: Byte, evalE2: Byte) => (evalE1 | evalE2).toByte).asInstanceOf[(Any, Any) => Any]
-    case ShortType =>
-      ((evalE1: Short, evalE2: Short) => (evalE1 | evalE2).toShort).asInstanceOf[(Any, Any) => Any]
-    case IntegerType =>
-      ((evalE1: Int, evalE2: Int) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any]
-    case LongType =>
-      ((evalE1: Long, evalE2: Long) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any]
-  }
-
-  protected override def evalInternal(evalE1: Any, evalE2: Any) = or(evalE1, evalE2)
-}
-
-/**
- * A function that calculates bitwise xor of two numbers.
- */
-case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic {
-  override def symbol: String = "^"
-
-  protected def checkTypesInternal(t: DataType) =
-    TypeUtils.checkForBitwiseExpr(t, "operator " + symbol)
-
-  private lazy val xor: (Any, Any) => Any = dataType match {
-    case ByteType =>
-      ((evalE1: Byte, evalE2: Byte) => (evalE1 ^ evalE2).toByte).asInstanceOf[(Any, Any) => Any]
-    case ShortType =>
-      ((evalE1: Short, evalE2: Short) => (evalE1 ^ evalE2).toShort).asInstanceOf[(Any, Any) => Any]
-    case IntegerType =>
-      ((evalE1: Int, evalE2: Int) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any]
-    case LongType =>
-      ((evalE1: Long, evalE2: Long) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any]
-  }
-
-  protected override def evalInternal(evalE1: Any, evalE2: Any): Any = xor(evalE1, evalE2)
-}
-
-/**
- * A function that calculates bitwise not(~) of a number.
- */
-case class BitwiseNot(child: Expression) extends UnaryArithmetic {
-  override def toString: String = s"~$child"
-
-  override def checkInputDataTypes(): TypeCheckResult =
-    TypeUtils.checkForBitwiseExpr(child.dataType, "operator ~")
-
-  private lazy val not: (Any) => Any = dataType match {
-    case ByteType =>
-      ((evalE: Byte) => (~evalE).toByte).asInstanceOf[(Any) => Any]
-    case ShortType =>
-      ((evalE: Short) => (~evalE).toShort).asInstanceOf[(Any) => Any]
-    case IntegerType =>
-      ((evalE: Int) => ~evalE).asInstanceOf[(Any) => Any]
-    case LongType =>
-      ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any]
-  }
-
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)})~($c)")
-  }
-
-  protected override def evalInternal(evalE: Any) = not(evalE)
-}
-
 case class MaxOf(left: Expression, right: Expression) extends BinaryArithmetic {
   override def nullable: Boolean = left.nullable && right.nullable
 
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ef34586261e7062ee8d30e44cfb8748151ce3df6
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwise.scala
@@ -0,0 +1,120 @@
+/*
+ * 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.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.util.TypeUtils
+import org.apache.spark.sql.types._
+
+
+/**
+ * A function that calculates bitwise and(&) of two numbers.
+ */
+case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def symbol: String = "&"
+
+  protected def checkTypesInternal(t: DataType) =
+    TypeUtils.checkForBitwiseExpr(t, "operator " + symbol)
+
+  private lazy val and: (Any, Any) => Any = dataType match {
+    case ByteType =>
+      ((evalE1: Byte, evalE2: Byte) => (evalE1 & evalE2).toByte).asInstanceOf[(Any, Any) => Any]
+    case ShortType =>
+      ((evalE1: Short, evalE2: Short) => (evalE1 & evalE2).toShort).asInstanceOf[(Any, Any) => Any]
+    case IntegerType =>
+      ((evalE1: Int, evalE2: Int) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any]
+    case LongType =>
+      ((evalE1: Long, evalE2: Long) => evalE1 & evalE2).asInstanceOf[(Any, Any) => Any]
+  }
+
+  protected override def evalInternal(evalE1: Any, evalE2: Any) = and(evalE1, evalE2)
+}
+
+/**
+ * A function that calculates bitwise or(|) of two numbers.
+ */
+case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def symbol: String = "|"
+
+  protected def checkTypesInternal(t: DataType) =
+    TypeUtils.checkForBitwiseExpr(t, "operator " + symbol)
+
+  private lazy val or: (Any, Any) => Any = dataType match {
+    case ByteType =>
+      ((evalE1: Byte, evalE2: Byte) => (evalE1 | evalE2).toByte).asInstanceOf[(Any, Any) => Any]
+    case ShortType =>
+      ((evalE1: Short, evalE2: Short) => (evalE1 | evalE2).toShort).asInstanceOf[(Any, Any) => Any]
+    case IntegerType =>
+      ((evalE1: Int, evalE2: Int) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any]
+    case LongType =>
+      ((evalE1: Long, evalE2: Long) => evalE1 | evalE2).asInstanceOf[(Any, Any) => Any]
+  }
+
+  protected override def evalInternal(evalE1: Any, evalE2: Any) = or(evalE1, evalE2)
+}
+
+/**
+ * A function that calculates bitwise xor of two numbers.
+ */
+case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic {
+  override def symbol: String = "^"
+
+  protected def checkTypesInternal(t: DataType) =
+    TypeUtils.checkForBitwiseExpr(t, "operator " + symbol)
+
+  private lazy val xor: (Any, Any) => Any = dataType match {
+    case ByteType =>
+      ((evalE1: Byte, evalE2: Byte) => (evalE1 ^ evalE2).toByte).asInstanceOf[(Any, Any) => Any]
+    case ShortType =>
+      ((evalE1: Short, evalE2: Short) => (evalE1 ^ evalE2).toShort).asInstanceOf[(Any, Any) => Any]
+    case IntegerType =>
+      ((evalE1: Int, evalE2: Int) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any]
+    case LongType =>
+      ((evalE1: Long, evalE2: Long) => evalE1 ^ evalE2).asInstanceOf[(Any, Any) => Any]
+  }
+
+  protected override def evalInternal(evalE1: Any, evalE2: Any): Any = xor(evalE1, evalE2)
+}
+
+/**
+ * A function that calculates bitwise not(~) of a number.
+ */
+case class BitwiseNot(child: Expression) extends UnaryArithmetic {
+  override def toString: String = s"~$child"
+
+  override def checkInputDataTypes(): TypeCheckResult =
+    TypeUtils.checkForBitwiseExpr(child.dataType, "operator ~")
+
+  private lazy val not: (Any) => Any = dataType match {
+    case ByteType =>
+      ((evalE: Byte) => (~evalE).toByte).asInstanceOf[(Any) => Any]
+    case ShortType =>
+      ((evalE: Short) => (~evalE).toShort).asInstanceOf[(Any) => Any]
+    case IntegerType =>
+      ((evalE: Int) => ~evalE).asInstanceOf[(Any) => Any]
+    case LongType =>
+      ((evalE: Long) => ~evalE).asInstanceOf[(Any) => Any]
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dataType)})~($c)")
+  }
+
+  protected override def evalInternal(evalE: Any) = not(evalE)
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala
new file mode 100644
index 0000000000000000000000000000000000000000..3aa86edd7ab205e653482714c32ad589fbc70063
--- /dev/null
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala
@@ -0,0 +1,313 @@
+/*
+ * 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.catalyst.expressions.codegen._
+import org.apache.spark.sql.types.{BooleanType, DataType}
+
+
+case class If(predicate: Expression, trueValue: Expression, falseValue: Expression)
+  extends Expression {
+
+  override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil
+  override def nullable: Boolean = trueValue.nullable || falseValue.nullable
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    if (predicate.dataType != BooleanType) {
+      TypeCheckResult.TypeCheckFailure(
+        s"type of predicate expression in If should be boolean, not ${predicate.dataType}")
+    } else if (trueValue.dataType != falseValue.dataType) {
+      TypeCheckResult.TypeCheckFailure(
+        s"differing types in If (${trueValue.dataType} and ${falseValue.dataType}).")
+    } else {
+      TypeCheckResult.TypeCheckSuccess
+    }
+  }
+
+  override def dataType: DataType = trueValue.dataType
+
+  override def eval(input: Row): Any = {
+    if (true == predicate.eval(input)) {
+      trueValue.eval(input)
+    } else {
+      falseValue.eval(input)
+    }
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    val condEval = predicate.gen(ctx)
+    val trueEval = trueValue.gen(ctx)
+    val falseEval = falseValue.gen(ctx)
+
+    s"""
+      ${condEval.code}
+      boolean ${ev.isNull} = false;
+      ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
+      if (!${condEval.isNull} && ${condEval.primitive}) {
+        ${trueEval.code}
+        ${ev.isNull} = ${trueEval.isNull};
+        ${ev.primitive} = ${trueEval.primitive};
+      } else {
+        ${falseEval.code}
+        ${ev.isNull} = ${falseEval.isNull};
+        ${ev.primitive} = ${falseEval.primitive};
+      }
+    """
+  }
+
+  override def toString: String = s"if ($predicate) $trueValue else $falseValue"
+}
+
+trait CaseWhenLike extends Expression {
+  self: Product =>
+
+  // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last
+  // element is the value for the default catch-all case (if provided).
+  // Hence, `branches` consists of at least two elements, and can have an odd or even length.
+  def branches: Seq[Expression]
+
+  @transient lazy val whenList =
+    branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq
+  @transient lazy val thenList =
+    branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq
+  val elseValue = if (branches.length % 2 == 0) None else Option(branches.last)
+
+  // both then and else expressions should be considered.
+  def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType)
+  def valueTypesEqual: Boolean = valueTypes.distinct.size == 1
+
+  override def checkInputDataTypes(): TypeCheckResult = {
+    if (valueTypesEqual) {
+      checkTypesInternal()
+    } else {
+      TypeCheckResult.TypeCheckFailure(
+        "THEN and ELSE expressions should all be same type or coercible to a common type")
+    }
+  }
+
+  protected def checkTypesInternal(): TypeCheckResult
+
+  override def dataType: DataType = thenList.head.dataType
+
+  override def nullable: Boolean = {
+    // If no value is nullable and no elseValue is provided, the whole statement defaults to null.
+    thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true))
+  }
+}
+
+// scalastyle:off
+/**
+ * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END".
+ * Refer to this link for the corresponding semantics:
+ * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions
+ */
+// scalastyle:on
+case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike {
+
+  // Use private[this] Array to speed up evaluation.
+  @transient private[this] lazy val branchesArr = branches.toArray
+
+  override def children: Seq[Expression] = branches
+
+  override protected def checkTypesInternal(): TypeCheckResult = {
+    if (whenList.forall(_.dataType == BooleanType)) {
+      TypeCheckResult.TypeCheckSuccess
+    } else {
+      val index = whenList.indexWhere(_.dataType != BooleanType)
+      TypeCheckResult.TypeCheckFailure(
+        s"WHEN expressions in CaseWhen should all be boolean type, " +
+          s"but the ${index + 1}th when expression's type is ${whenList(index)}")
+    }
+  }
+
+  /** Written in imperative fashion for performance considerations. */
+  override def eval(input: Row): Any = {
+    val len = branchesArr.length
+    var i = 0
+    // If all branches fail and an elseVal is not provided, the whole statement
+    // defaults to null, according to Hive's semantics.
+    while (i < len - 1) {
+      if (branchesArr(i).eval(input) == true) {
+        return branchesArr(i + 1).eval(input)
+      }
+      i += 2
+    }
+    var res: Any = null
+    if (i == len - 1) {
+      res = branchesArr(i).eval(input)
+    }
+    return res
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    val len = branchesArr.length
+    val got = ctx.freshName("got")
+
+    val cases = (0 until len/2).map { i =>
+      val cond = branchesArr(i * 2).gen(ctx)
+      val res = branchesArr(i * 2 + 1).gen(ctx)
+      s"""
+        if (!$got) {
+          ${cond.code}
+          if (!${cond.isNull} && ${cond.primitive}) {
+            $got = true;
+            ${res.code}
+            ${ev.isNull} = ${res.isNull};
+            ${ev.primitive} = ${res.primitive};
+          }
+        }
+      """
+    }.mkString("\n")
+
+    val other = if (len % 2 == 1) {
+      val res = branchesArr(len - 1).gen(ctx)
+      s"""
+        if (!$got) {
+          ${res.code}
+          ${ev.isNull} = ${res.isNull};
+          ${ev.primitive} = ${res.primitive};
+        }
+      """
+    } else {
+      ""
+    }
+
+    s"""
+      boolean $got = false;
+      boolean ${ev.isNull} = true;
+      ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
+      $cases
+      $other
+    """
+  }
+
+  override def toString: String = {
+    "CASE" + branches.sliding(2, 2).map {
+      case Seq(cond, value) => s" WHEN $cond THEN $value"
+      case Seq(elseValue) => s" ELSE $elseValue"
+    }.mkString
+  }
+}
+
+// scalastyle:off
+/**
+ * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END".
+ * Refer to this link for the corresponding semantics:
+ * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions
+ */
+// scalastyle:on
+case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike {
+
+  // Use private[this] Array to speed up evaluation.
+  @transient private[this] lazy val branchesArr = branches.toArray
+
+  override def children: Seq[Expression] = key +: branches
+
+  override protected def checkTypesInternal(): TypeCheckResult = {
+    if ((key +: whenList).map(_.dataType).distinct.size > 1) {
+      TypeCheckResult.TypeCheckFailure(
+        "key and WHEN expressions should all be same type or coercible to a common type")
+    } else {
+      TypeCheckResult.TypeCheckSuccess
+    }
+  }
+
+  /** Written in imperative fashion for performance considerations. */
+  override def eval(input: Row): Any = {
+    val evaluatedKey = key.eval(input)
+    val len = branchesArr.length
+    var i = 0
+    // If all branches fail and an elseVal is not provided, the whole statement
+    // defaults to null, according to Hive's semantics.
+    while (i < len - 1) {
+      if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) {
+        return branchesArr(i + 1).eval(input)
+      }
+      i += 2
+    }
+    var res: Any = null
+    if (i == len - 1) {
+      res = branchesArr(i).eval(input)
+    }
+    return res
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    val keyEval = key.gen(ctx)
+    val len = branchesArr.length
+    val got = ctx.freshName("got")
+
+    val cases = (0 until len/2).map { i =>
+      val cond = branchesArr(i * 2).gen(ctx)
+      val res = branchesArr(i * 2 + 1).gen(ctx)
+      s"""
+        if (!$got) {
+          ${cond.code}
+          if (${keyEval.isNull} && ${cond.isNull} ||
+            !${keyEval.isNull} && !${cond.isNull}
+             && ${ctx.equalFunc(key.dataType)(keyEval.primitive, cond.primitive)}) {
+            $got = true;
+            ${res.code}
+            ${ev.isNull} = ${res.isNull};
+            ${ev.primitive} = ${res.primitive};
+          }
+        }
+      """
+    }.mkString("\n")
+
+    val other = if (len % 2 == 1) {
+      val res = branchesArr(len - 1).gen(ctx)
+      s"""
+        if (!$got) {
+          ${res.code}
+          ${ev.isNull} = ${res.isNull};
+          ${ev.primitive} = ${res.primitive};
+        }
+      """
+    } else {
+      ""
+    }
+
+    s"""
+      boolean $got = false;
+      boolean ${ev.isNull} = true;
+      ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
+      ${keyEval.code}
+      $cases
+      $other
+    """
+  }
+
+  private def equalNullSafe(l: Any, r: Any) = {
+    if (l == null && r == null) {
+      true
+    } else if (l == null || r == null) {
+      false
+    } else {
+      l == r
+    }
+  }
+
+  override def toString: String = {
+    s"CASE $key" + branches.sliding(2, 2).map {
+      case Seq(cond, value) => s" WHEN $cond THEN $value"
+      case Seq(elseValue) => s" ELSE $elseValue"
+    }.mkString
+  }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
similarity index 54%
rename from sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala
rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
index 5563cd94bf86d653e9299f58819b4e0b1091c514..a18067e4a58f1288e4bf344c69a24ba15c2cd314 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/unary.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala
@@ -15,11 +15,10 @@
  * limitations under the License.
  */
 
-package org.apache.spark.sql.catalyst.expressions.mathfuncs
+package org.apache.spark.sql.catalyst.expressions
 
-import org.apache.spark.sql.catalyst.expressions.codegen.{Code, CodeGenContext, GeneratedExpressionCode}
-import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, Row, UnaryExpression}
-import org.apache.spark.sql.types._
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.types.{DataType, DoubleType}
 
 /**
  * A unary expression specifically for math functions. Math Functions expect a specific type of
@@ -64,6 +63,47 @@ abstract class UnaryMathExpression(f: Double => Double, name: String)
   }
 }
 
+/**
+ * A binary expression specifically for math functions that take two `Double`s as input and returns
+ * a `Double`.
+ * @param f The math function.
+ * @param name The short name of the function
+ */
+abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String)
+  extends BinaryExpression with Serializable with ExpectsInputTypes { self: Product =>
+
+  override def expectedChildTypes: Seq[DataType] = Seq(DoubleType, DoubleType)
+
+  override def toString: String = s"$name($left, $right)"
+
+  override def dataType: DataType = DoubleType
+
+  override def eval(input: Row): Any = {
+    val evalE1 = left.eval(input)
+    if (evalE1 == null) {
+      null
+    } else {
+      val evalE2 = right.eval(input)
+      if (evalE2 == null) {
+        null
+      } else {
+        val result = f(evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double])
+        if (result.isNaN) null else result
+      }
+    }
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)")
+  }
+}
+
+////////////////////////////////////////////////////////////////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////////////////////////////////
+// Unary math functions
+////////////////////////////////////////////////////////////////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////////////////////////////////
+
 case class Acos(child: Expression) extends UnaryMathExpression(math.acos, "ACOS")
 
 case class Asin(child: Expression) extends UnaryMathExpression(math.asin, "ASIN")
@@ -111,3 +151,54 @@ case class ToDegrees(child: Expression) extends UnaryMathExpression(math.toDegre
 case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadians, "RADIANS") {
   override def funcName: String = "toRadians"
 }
+
+
+////////////////////////////////////////////////////////////////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////////////////////////////////
+// Binary math functions
+////////////////////////////////////////////////////////////////////////////////////////////////////
+////////////////////////////////////////////////////////////////////////////////////////////////////
+
+
+case class Atan2(left: Expression, right: Expression)
+  extends BinaryMathExpression(math.atan2, "ATAN2") {
+
+  override def eval(input: Row): Any = {
+    val evalE1 = left.eval(input)
+    if (evalE1 == null) {
+      null
+    } else {
+      val evalE2 = right.eval(input)
+      if (evalE2 == null) {
+        null
+      } else {
+        // With codegen, the values returned by -0.0 and 0.0 are different. Handled with +0.0
+        val result = math.atan2(evalE1.asInstanceOf[Double] + 0.0,
+          evalE2.asInstanceOf[Double] + 0.0)
+        if (result.isNaN) null else result
+      }
+    }
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") + s"""
+      if (Double.valueOf(${ev.primitive}).isNaN()) {
+        ${ev.isNull} = true;
+      }
+      """
+  }
+}
+
+case class Hypot(left: Expression, right: Expression)
+  extends BinaryMathExpression(math.hypot, "HYPOT")
+
+case class Pow(left: Expression, right: Expression)
+  extends BinaryMathExpression(math.pow, "POWER") {
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
+    defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") + s"""
+      if (Double.valueOf(${ev.primitive}).isNaN()) {
+        ${ev.isNull} = true;
+      }
+      """
+  }
+}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala
deleted file mode 100644
index 88211acd7713c13e4c0ebfaac46a7d68e440fd7a..0000000000000000000000000000000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/mathfuncs/binary.scala
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * 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.mathfuncs
-
-import org.apache.spark.sql.catalyst.expressions.codegen._
-import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, BinaryExpression, Expression, Row}
-import org.apache.spark.sql.types._
-
-/**
- * A binary expression specifically for math functions that take two `Double`s as input and returns
- * a `Double`.
- * @param f The math function.
- * @param name The short name of the function
- */
-abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String)
-  extends BinaryExpression with Serializable with ExpectsInputTypes { self: Product =>
-
-  override def expectedChildTypes: Seq[DataType] = Seq(DoubleType, DoubleType)
-
-  override def toString: String = s"$name($left, $right)"
-
-  override def dataType: DataType = DoubleType
-
-  override def eval(input: Row): Any = {
-    val evalE1 = left.eval(input)
-    if (evalE1 == null) {
-      null
-    } else {
-      val evalE2 = right.eval(input)
-      if (evalE2 == null) {
-        null
-      } else {
-        val result = f(evalE1.asInstanceOf[Double], evalE2.asInstanceOf[Double])
-        if (result.isNaN) null else result
-      }
-    }
-  }
-
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.${name.toLowerCase}($c1, $c2)")
-  }
-}
-
-case class Atan2(left: Expression, right: Expression)
-  extends BinaryMathExpression(math.atan2, "ATAN2") {
-
-  override def eval(input: Row): Any = {
-    val evalE1 = left.eval(input)
-    if (evalE1 == null) {
-      null
-    } else {
-      val evalE2 = right.eval(input)
-      if (evalE2 == null) {
-        null
-      } else {
-        // With codegen, the values returned by -0.0 and 0.0 are different. Handled with +0.0
-        val result = math.atan2(evalE1.asInstanceOf[Double] + 0.0,
-          evalE2.asInstanceOf[Double] + 0.0)
-        if (result.isNaN) null else result
-      }
-    }
-  }
-
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.atan2($c1 + 0.0, $c2 + 0.0)") + s"""
-      if (Double.valueOf(${ev.primitive}).isNaN()) {
-        ${ev.isNull} = true;
-      }
-      """
-  }
-}
-
-case class Hypot(left: Expression, right: Expression)
-  extends BinaryMathExpression(math.hypot, "HYPOT")
-
-case class Pow(left: Expression, right: Expression)
-  extends BinaryMathExpression(math.pow, "POWER") {
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.pow($c1, $c2)") + s"""
-      if (Double.valueOf(${ev.primitive}).isNaN()) {
-        ${ev.isNull} = true;
-      }
-      """
-  }
-}
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 1d0f19a400d638019e819180f2c934cdb721049f..5edcf3bd77d201fd7117f724cf9601bbcf13331e 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
@@ -359,293 +359,3 @@ case class GreaterThanOrEqual(left: Expression, right: Expression) extends Binar
 
   protected override def evalInternal(evalE1: Any, evalE2: Any) = ordering.gteq(evalE1, evalE2)
 }
-
-case class If(predicate: Expression, trueValue: Expression, falseValue: Expression)
-  extends Expression {
-
-  override def children: Seq[Expression] = predicate :: trueValue :: falseValue :: Nil
-  override def nullable: Boolean = trueValue.nullable || falseValue.nullable
-
-  override def checkInputDataTypes(): TypeCheckResult = {
-    if (predicate.dataType != BooleanType) {
-      TypeCheckResult.TypeCheckFailure(
-        s"type of predicate expression in If should be boolean, not ${predicate.dataType}")
-    } else if (trueValue.dataType != falseValue.dataType) {
-      TypeCheckResult.TypeCheckFailure(
-        s"differing types in If (${trueValue.dataType} and ${falseValue.dataType}).")
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
-  }
-
-  override def dataType: DataType = trueValue.dataType
-
-  override def eval(input: Row): Any = {
-    if (true == predicate.eval(input)) {
-      trueValue.eval(input)
-    } else {
-      falseValue.eval(input)
-    }
-  }
-
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    val condEval = predicate.gen(ctx)
-    val trueEval = trueValue.gen(ctx)
-    val falseEval = falseValue.gen(ctx)
-
-    s"""
-      ${condEval.code}
-      boolean ${ev.isNull} = false;
-      ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
-      if (!${condEval.isNull} && ${condEval.primitive}) {
-        ${trueEval.code}
-        ${ev.isNull} = ${trueEval.isNull};
-        ${ev.primitive} = ${trueEval.primitive};
-      } else {
-        ${falseEval.code}
-        ${ev.isNull} = ${falseEval.isNull};
-        ${ev.primitive} = ${falseEval.primitive};
-      }
-    """
-  }
-
-  override def toString: String = s"if ($predicate) $trueValue else $falseValue"
-}
-
-trait CaseWhenLike extends Expression {
-  self: Product =>
-
-  // Note that `branches` are considered in consecutive pairs (cond, val), and the optional last
-  // element is the value for the default catch-all case (if provided).
-  // Hence, `branches` consists of at least two elements, and can have an odd or even length.
-  def branches: Seq[Expression]
-
-  @transient lazy val whenList =
-    branches.sliding(2, 2).collect { case Seq(whenExpr, _) => whenExpr }.toSeq
-  @transient lazy val thenList =
-    branches.sliding(2, 2).collect { case Seq(_, thenExpr) => thenExpr }.toSeq
-  val elseValue = if (branches.length % 2 == 0) None else Option(branches.last)
-
-  // both then and else expressions should be considered.
-  def valueTypes: Seq[DataType] = (thenList ++ elseValue).map(_.dataType)
-  def valueTypesEqual: Boolean = valueTypes.distinct.size == 1
-
-  override def checkInputDataTypes(): TypeCheckResult = {
-    if (valueTypesEqual) {
-      checkTypesInternal()
-    } else {
-      TypeCheckResult.TypeCheckFailure(
-        "THEN and ELSE expressions should all be same type or coercible to a common type")
-    }
-  }
-
-  protected def checkTypesInternal(): TypeCheckResult
-
-  override def dataType: DataType = thenList.head.dataType
-
-  override def nullable: Boolean = {
-    // If no value is nullable and no elseValue is provided, the whole statement defaults to null.
-    thenList.exists(_.nullable) || (elseValue.map(_.nullable).getOrElse(true))
-  }
-}
-
-// scalastyle:off
-/**
- * Case statements of the form "CASE WHEN a THEN b [WHEN c THEN d]* [ELSE e] END".
- * Refer to this link for the corresponding semantics:
- * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions
- */
-// scalastyle:on
-case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike {
-
-  // Use private[this] Array to speed up evaluation.
-  @transient private[this] lazy val branchesArr = branches.toArray
-
-  override def children: Seq[Expression] = branches
-
-  override protected def checkTypesInternal(): TypeCheckResult = {
-    if (whenList.forall(_.dataType == BooleanType)) {
-      TypeCheckResult.TypeCheckSuccess
-    } else {
-      val index = whenList.indexWhere(_.dataType != BooleanType)
-      TypeCheckResult.TypeCheckFailure(
-        s"WHEN expressions in CaseWhen should all be boolean type, " +
-        s"but the ${index + 1}th when expression's type is ${whenList(index)}")
-    }
-  }
-
-  /** Written in imperative fashion for performance considerations. */
-  override def eval(input: Row): Any = {
-    val len = branchesArr.length
-    var i = 0
-    // If all branches fail and an elseVal is not provided, the whole statement
-    // defaults to null, according to Hive's semantics.
-    while (i < len - 1) {
-      if (branchesArr(i).eval(input) == true) {
-        return branchesArr(i + 1).eval(input)
-      }
-      i += 2
-    }
-    var res: Any = null
-    if (i == len - 1) {
-      res = branchesArr(i).eval(input)
-    }
-    return res
-  }
-
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    val len = branchesArr.length
-    val got = ctx.freshName("got")
-
-    val cases = (0 until len/2).map { i =>
-      val cond = branchesArr(i * 2).gen(ctx)
-      val res = branchesArr(i * 2 + 1).gen(ctx)
-      s"""
-        if (!$got) {
-          ${cond.code}
-          if (!${cond.isNull} && ${cond.primitive}) {
-            $got = true;
-            ${res.code}
-            ${ev.isNull} = ${res.isNull};
-            ${ev.primitive} = ${res.primitive};
-          }
-        }
-      """
-    }.mkString("\n")
-
-    val other = if (len % 2 == 1) {
-      val res = branchesArr(len - 1).gen(ctx)
-      s"""
-        if (!$got) {
-          ${res.code}
-          ${ev.isNull} = ${res.isNull};
-          ${ev.primitive} = ${res.primitive};
-        }
-      """
-    } else {
-      ""
-    }
-
-    s"""
-      boolean $got = false;
-      boolean ${ev.isNull} = true;
-      ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
-      $cases
-      $other
-    """
-  }
-
-  override def toString: String = {
-    "CASE" + branches.sliding(2, 2).map {
-      case Seq(cond, value) => s" WHEN $cond THEN $value"
-      case Seq(elseValue) => s" ELSE $elseValue"
-    }.mkString
-  }
-}
-
-// scalastyle:off
-/**
- * Case statements of the form "CASE a WHEN b THEN c [WHEN d THEN e]* [ELSE f] END".
- * Refer to this link for the corresponding semantics:
- * https://cwiki.apache.org/confluence/display/Hive/LanguageManual+UDF#LanguageManualUDF-ConditionalFunctions
- */
-// scalastyle:on
-case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseWhenLike {
-
-  // Use private[this] Array to speed up evaluation.
-  @transient private[this] lazy val branchesArr = branches.toArray
-
-  override def children: Seq[Expression] = key +: branches
-
-  override protected def checkTypesInternal(): TypeCheckResult = {
-    if ((key +: whenList).map(_.dataType).distinct.size > 1) {
-      TypeCheckResult.TypeCheckFailure(
-        "key and WHEN expressions should all be same type or coercible to a common type")
-    } else {
-      TypeCheckResult.TypeCheckSuccess
-    }
-  }
-
-  /** Written in imperative fashion for performance considerations. */
-  override def eval(input: Row): Any = {
-    val evaluatedKey = key.eval(input)
-    val len = branchesArr.length
-    var i = 0
-    // If all branches fail and an elseVal is not provided, the whole statement
-    // defaults to null, according to Hive's semantics.
-    while (i < len - 1) {
-      if (equalNullSafe(evaluatedKey, branchesArr(i).eval(input))) {
-        return branchesArr(i + 1).eval(input)
-      }
-      i += 2
-    }
-    var res: Any = null
-    if (i == len - 1) {
-      res = branchesArr(i).eval(input)
-    }
-    return res
-  }
-
-  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): Code = {
-    val keyEval = key.gen(ctx)
-    val len = branchesArr.length
-    val got = ctx.freshName("got")
-
-    val cases = (0 until len/2).map { i =>
-      val cond = branchesArr(i * 2).gen(ctx)
-      val res = branchesArr(i * 2 + 1).gen(ctx)
-      s"""
-        if (!$got) {
-          ${cond.code}
-          if (${keyEval.isNull} && ${cond.isNull} ||
-            !${keyEval.isNull} && !${cond.isNull}
-             && ${ctx.equalFunc(key.dataType)(keyEval.primitive, cond.primitive)}) {
-            $got = true;
-            ${res.code}
-            ${ev.isNull} = ${res.isNull};
-            ${ev.primitive} = ${res.primitive};
-          }
-        }
-      """
-    }.mkString("\n")
-
-    val other = if (len % 2 == 1) {
-      val res = branchesArr(len - 1).gen(ctx)
-      s"""
-        if (!$got) {
-          ${res.code}
-          ${ev.isNull} = ${res.isNull};
-          ${ev.primitive} = ${res.primitive};
-        }
-      """
-    } else {
-      ""
-    }
-
-    s"""
-      boolean $got = false;
-      boolean ${ev.isNull} = true;
-      ${ctx.javaType(dataType)} ${ev.primitive} = ${ctx.defaultValue(dataType)};
-      ${keyEval.code}
-      $cases
-      $other
-    """
-  }
-
-  private def equalNullSafe(l: Any, r: Any) = {
-    if (l == null && r == null) {
-      true
-    } else if (l == null || r == null) {
-      false
-    } else {
-      l == r
-    }
-  }
-
-  override def toString: String = {
-    s"CASE $key" + branches.sliding(2, 2).map {
-      case Seq(cond, value) => s" WHEN $cond THEN $value"
-      case Seq(elseValue) => s" ELSE $elseValue"
-    }.mkString
-  }
-}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..e1afa81a7a82f338ddfc1ce13a387e9f64a4ffb0
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala
@@ -0,0 +1,144 @@
+/*
+ * 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.scalatest.Matchers._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types.{DoubleType, IntegerType}
+
+
+class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  test("arithmetic") {
+    val row = create_row(1, 2, 3, null)
+    val c1 = 'a.int.at(0)
+    val c2 = 'a.int.at(1)
+    val c3 = 'a.int.at(2)
+    val c4 = 'a.int.at(3)
+
+    checkEvaluation(UnaryMinus(c1), -1, row)
+    checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100)
+
+    checkEvaluation(Add(c1, c4), null, row)
+    checkEvaluation(Add(c1, c2), 3, row)
+    checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row)
+    checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row)
+    checkEvaluation(
+      Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
+
+    checkEvaluation(-c1, -1, row)
+    checkEvaluation(c1 + c2, 3, row)
+    checkEvaluation(c1 - c2, -1, row)
+    checkEvaluation(c1 * c2, 2, row)
+    checkEvaluation(c1 / c2, 0, row)
+    checkEvaluation(c1 % c2, 1, row)
+  }
+
+  test("fractional arithmetic") {
+    val row = create_row(1.1, 2.0, 3.1, null)
+    val c1 = 'a.double.at(0)
+    val c2 = 'a.double.at(1)
+    val c3 = 'a.double.at(2)
+    val c4 = 'a.double.at(3)
+
+    checkEvaluation(UnaryMinus(c1), -1.1, row)
+    checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0)
+    checkEvaluation(Add(c1, c4), null, row)
+    checkEvaluation(Add(c1, c2), 3.1, row)
+    checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row)
+    checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row)
+    checkEvaluation(
+      Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row)
+
+    checkEvaluation(-c1, -1.1, row)
+    checkEvaluation(c1 + c2, 3.1, row)
+    checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row)
+    checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row)
+    checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row)
+    checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row)
+  }
+
+  test("Divide") {
+    checkEvaluation(Divide(Literal(2), Literal(1)), 2)
+    checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5)
+    checkEvaluation(Divide(Literal(1), Literal(2)), 0)
+    checkEvaluation(Divide(Literal(1), Literal(0)), null)
+    checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null)
+    checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null)
+    checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null)
+    checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null)
+    checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null)
+    checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null)
+    checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null)
+    checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)),
+      null)
+  }
+
+  test("Remainder") {
+    checkEvaluation(Remainder(Literal(2), Literal(1)), 0)
+    checkEvaluation(Remainder(Literal(1.0), Literal(2.0)), 1.0)
+    checkEvaluation(Remainder(Literal(1), Literal(2)), 1)
+    checkEvaluation(Remainder(Literal(1), Literal(0)), null)
+    checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null)
+    checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null)
+    checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null)
+    checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null)
+    checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null)
+    checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null)
+    checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null)
+    checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)),
+      null)
+  }
+
+  test("MaxOf") {
+    checkEvaluation(MaxOf(1, 2), 2)
+    checkEvaluation(MaxOf(2, 1), 2)
+    checkEvaluation(MaxOf(1L, 2L), 2L)
+    checkEvaluation(MaxOf(2L, 1L), 2L)
+
+    checkEvaluation(MaxOf(Literal.create(null, IntegerType), 2), 2)
+    checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2)
+  }
+
+  test("MinOf") {
+    checkEvaluation(MinOf(1, 2), 1)
+    checkEvaluation(MinOf(2, 1), 1)
+    checkEvaluation(MinOf(1L, 2L), 1L)
+    checkEvaluation(MinOf(2L, 1L), 1L)
+
+    checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1)
+    checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1)
+  }
+
+  test("SQRT") {
+    val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24))
+    val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble))
+    val rowSequence = inputSequence.map(l => create_row(l.toDouble))
+    val d = 'a.double.at(0)
+
+    for ((row, expected) <- rowSequence zip expectedResults) {
+      checkEvaluation(Sqrt(d), expected, row)
+    }
+
+    checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null))
+    checkEvaluation(Sqrt(-1), null, EmptyRow)
+    checkEvaluation(Sqrt(-1.5), null, EmptyRow)
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..c9bbc7a8b8c146e9d6390fc5efeed8635118a68a
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/BitwiseFunctionsSuite.scala
@@ -0,0 +1,80 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types._
+
+
+class BitwiseFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  test("Bitwise operations") {
+    val row = create_row(1, 2, 3, null)
+    val c1 = 'a.int.at(0)
+    val c2 = 'a.int.at(1)
+    val c3 = 'a.int.at(2)
+    val c4 = 'a.int.at(3)
+
+    checkEvaluation(BitwiseAnd(c1, c4), null, row)
+    checkEvaluation(BitwiseAnd(c1, c2), 0, row)
+    checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row)
+    checkEvaluation(
+      BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
+
+    checkEvaluation(BitwiseOr(c1, c4), null, row)
+    checkEvaluation(BitwiseOr(c1, c2), 3, row)
+    checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row)
+    checkEvaluation(
+      BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
+
+    checkEvaluation(BitwiseXor(c1, c4), null, row)
+    checkEvaluation(BitwiseXor(c1, c2), 3, row)
+    checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row)
+    checkEvaluation(
+      BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
+
+    checkEvaluation(BitwiseNot(c4), null, row)
+    checkEvaluation(BitwiseNot(c1), -2, row)
+    checkEvaluation(BitwiseNot(Literal.create(null, IntegerType)), null, row)
+
+    checkEvaluation(c1 & c2, 0, row)
+    checkEvaluation(c1 | c2, 3, row)
+    checkEvaluation(c1 ^ c2, 3, row)
+    checkEvaluation(~c1, -2, row)
+  }
+
+  test("unary BitwiseNOT") {
+    checkEvaluation(BitwiseNot(1), -2)
+    assert(BitwiseNot(1).dataType === IntegerType)
+    assert(BitwiseNot(1).eval(EmptyRow).isInstanceOf[Int])
+
+    checkEvaluation(BitwiseNot(1.toLong), -2.toLong)
+    assert(BitwiseNot(1.toLong).dataType === LongType)
+    assert(BitwiseNot(1.toLong).eval(EmptyRow).isInstanceOf[Long])
+
+    checkEvaluation(BitwiseNot(1.toShort), -2.toShort)
+    assert(BitwiseNot(1.toShort).dataType === ShortType)
+    assert(BitwiseNot(1.toShort).eval(EmptyRow).isInstanceOf[Short])
+
+    checkEvaluation(BitwiseNot(1.toByte), -2.toByte)
+    assert(BitwiseNot(1.toByte).dataType === ByteType)
+    assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte])
+  }
+
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..5bc7c30eee1b60ea64c10a6b07b9832c8d135421
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala
@@ -0,0 +1,532 @@
+/*
+ * 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 java.sql.{Timestamp, Date}
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.types._
+
+/**
+ * Test suite for data type casting expression [[Cast]].
+ */
+class CastSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  private def cast(v: Any, targetType: DataType): Cast = {
+    v match {
+      case lit: Expression => Cast(lit, targetType)
+      case _ => Cast(Literal(v), targetType)
+    }
+  }
+
+  // expected cannot be null
+  private def checkCast(v: Any, expected: Any): Unit = {
+    checkEvaluation(cast(v, Literal(expected).dataType), expected)
+  }
+
+  test("cast from int") {
+    checkCast(0, false)
+    checkCast(1, true)
+    checkCast(5, true)
+    checkCast(1, 1.toByte)
+    checkCast(1, 1.toShort)
+    checkCast(1, 1)
+    checkCast(1, 1.toLong)
+    checkCast(1, 1.0f)
+    checkCast(1, 1.0)
+    checkCast(123, "123")
+
+    checkEvaluation(cast(123, DecimalType.Unlimited), Decimal(123))
+    checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123))
+    checkEvaluation(cast(123, DecimalType(3, 1)), null)
+    checkEvaluation(cast(123, DecimalType(2, 0)), null)
+  }
+
+  test("cast from long") {
+    checkCast(0L, false)
+    checkCast(1L, true)
+    checkCast(5L, true)
+    checkCast(1L, 1.toByte)
+    checkCast(1L, 1.toShort)
+    checkCast(1L, 1)
+    checkCast(1L, 1.toLong)
+    checkCast(1L, 1.0f)
+    checkCast(1L, 1.0)
+    checkCast(123L, "123")
+
+    checkEvaluation(cast(123L, DecimalType.Unlimited), Decimal(123))
+    checkEvaluation(cast(123L, DecimalType(3, 0)), Decimal(123))
+    checkEvaluation(cast(123L, DecimalType(3, 1)), Decimal(123.0))
+
+    // TODO: Fix the following bug and re-enable it.
+    // checkEvaluation(cast(123L, DecimalType(2, 0)), null)
+  }
+
+  test("cast from boolean") {
+    checkEvaluation(cast(true, IntegerType), 1)
+    checkEvaluation(cast(false, IntegerType), 0)
+    checkEvaluation(cast(true, StringType), "true")
+    checkEvaluation(cast(false, StringType), "false")
+    checkEvaluation(cast(cast(1, BooleanType), IntegerType), 1)
+    checkEvaluation(cast(cast(0, BooleanType), IntegerType), 0)
+  }
+
+  test("cast from int 2") {
+    checkEvaluation(cast(1, LongType), 1.toLong)
+    checkEvaluation(cast(cast(1000, TimestampType), LongType), 1.toLong)
+    checkEvaluation(cast(cast(-1200, TimestampType), LongType), -2.toLong)
+
+    checkEvaluation(cast(123, DecimalType.Unlimited), Decimal(123))
+    checkEvaluation(cast(123, DecimalType(3, 0)), Decimal(123))
+    checkEvaluation(cast(123, DecimalType(3, 1)), null)
+    checkEvaluation(cast(123, DecimalType(2, 0)), null)
+  }
+
+  test("cast from float") {
+
+  }
+
+  test("cast from double") {
+    checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble)
+    checkEvaluation(cast(cast(1.toDouble, TimestampType), DoubleType), 1.toDouble)
+  }
+
+  test("cast from string") {
+    assert(cast("abcdef", StringType).nullable === false)
+    assert(cast("abcdef", BinaryType).nullable === false)
+    assert(cast("abcdef", BooleanType).nullable === false)
+    assert(cast("abcdef", TimestampType).nullable === true)
+    assert(cast("abcdef", LongType).nullable === true)
+    assert(cast("abcdef", IntegerType).nullable === true)
+    assert(cast("abcdef", ShortType).nullable === true)
+    assert(cast("abcdef", ByteType).nullable === true)
+    assert(cast("abcdef", DecimalType.Unlimited).nullable === true)
+    assert(cast("abcdef", DecimalType(4, 2)).nullable === true)
+    assert(cast("abcdef", DoubleType).nullable === true)
+    assert(cast("abcdef", FloatType).nullable === true)
+  }
+
+  test("data type casting") {
+    val sd = "1970-01-01"
+    val d = Date.valueOf(sd)
+    val zts = sd + " 00:00:00"
+    val sts = sd + " 00:00:02"
+    val nts = sts + ".1"
+    val ts = Timestamp.valueOf(nts)
+
+    checkEvaluation(cast("abdef", StringType), "abdef")
+    checkEvaluation(cast("abdef", DecimalType.Unlimited), null)
+    checkEvaluation(cast("abdef", TimestampType), null)
+    checkEvaluation(cast("12.65", DecimalType.Unlimited), Decimal(12.65))
+
+    checkEvaluation(cast(cast(sd, DateType), StringType), sd)
+    checkEvaluation(cast(cast(d, StringType), DateType), 0)
+    checkEvaluation(cast(cast(nts, TimestampType), StringType), nts)
+    checkEvaluation(cast(cast(ts, StringType), TimestampType), ts)
+
+    // all convert to string type to check
+    checkEvaluation(cast(cast(cast(nts, TimestampType), DateType), StringType), sd)
+    checkEvaluation(cast(cast(cast(ts, DateType), TimestampType), StringType), zts)
+
+    checkEvaluation(cast(cast("abdef", BinaryType), StringType), "abdef")
+
+    checkEvaluation(cast(cast(cast(cast(
+      cast(cast("5", ByteType), ShortType), IntegerType), FloatType), DoubleType), LongType),
+      5.toLong)
+    checkEvaluation(
+      cast(cast(cast(cast(cast(cast("5", ByteType), TimestampType),
+        DecimalType.Unlimited), LongType), StringType), ShortType),
+      0.toShort)
+    checkEvaluation(
+      cast(cast(cast(cast(cast(cast("5", TimestampType), ByteType),
+        DecimalType.Unlimited), LongType), StringType), ShortType),
+      null)
+    checkEvaluation(cast(cast(cast(cast(cast(cast("5", DecimalType.Unlimited),
+      ByteType), TimestampType), LongType), StringType), ShortType),
+      0.toShort)
+
+    checkEvaluation(cast("23", DoubleType), 23d)
+    checkEvaluation(cast("23", IntegerType), 23)
+    checkEvaluation(cast("23", FloatType), 23f)
+    checkEvaluation(cast("23", DecimalType.Unlimited), Decimal(23))
+    checkEvaluation(cast("23", ByteType), 23.toByte)
+    checkEvaluation(cast("23", ShortType), 23.toShort)
+    checkEvaluation(cast("2012-12-11", DoubleType), null)
+    checkEvaluation(cast(123, IntegerType), 123)
+
+
+    checkEvaluation(cast(Literal.create(null, IntegerType), ShortType), null)
+  }
+
+  test("cast and add") {
+    checkEvaluation(Add(Literal(23d), cast(true, DoubleType)), 24d)
+    checkEvaluation(Add(Literal(23), cast(true, IntegerType)), 24)
+    checkEvaluation(Add(Literal(23f), cast(true, FloatType)), 24f)
+    checkEvaluation(Add(Literal(Decimal(23)), cast(true, DecimalType.Unlimited)), Decimal(24))
+    checkEvaluation(Add(Literal(23.toByte), cast(true, ByteType)), 24.toByte)
+    checkEvaluation(Add(Literal(23.toShort), cast(true, ShortType)), 24.toShort)
+  }
+
+  test("casting to fixed-precision decimals") {
+    // Overflow and rounding for casting to fixed-precision decimals:
+    // - Values should round with HALF_UP mode by default when you lower scale
+    // - Values that would overflow the target precision should turn into null
+    // - Because of this, casts to fixed-precision decimals should be nullable
+
+    assert(cast(123, DecimalType.Unlimited).nullable === false)
+    assert(cast(10.03f, DecimalType.Unlimited).nullable === true)
+    assert(cast(10.03, DecimalType.Unlimited).nullable === true)
+    assert(cast(Decimal(10.03), DecimalType.Unlimited).nullable === false)
+
+    assert(cast(123, DecimalType(2, 1)).nullable === true)
+    assert(cast(10.03f, DecimalType(2, 1)).nullable === true)
+    assert(cast(10.03, DecimalType(2, 1)).nullable === true)
+    assert(cast(Decimal(10.03), DecimalType(2, 1)).nullable === true)
+
+
+    checkEvaluation(cast(10.03, DecimalType.Unlimited), Decimal(10.03))
+    checkEvaluation(cast(10.03, DecimalType(4, 2)), Decimal(10.03))
+    checkEvaluation(cast(10.03, DecimalType(3, 1)), Decimal(10.0))
+    checkEvaluation(cast(10.03, DecimalType(2, 0)), Decimal(10))
+    checkEvaluation(cast(10.03, DecimalType(1, 0)), null)
+    checkEvaluation(cast(10.03, DecimalType(2, 1)), null)
+    checkEvaluation(cast(10.03, DecimalType(3, 2)), null)
+    checkEvaluation(cast(Decimal(10.03), DecimalType(3, 1)), Decimal(10.0))
+    checkEvaluation(cast(Decimal(10.03), DecimalType(3, 2)), null)
+
+    checkEvaluation(cast(10.05, DecimalType.Unlimited), Decimal(10.05))
+    checkEvaluation(cast(10.05, DecimalType(4, 2)), Decimal(10.05))
+    checkEvaluation(cast(10.05, DecimalType(3, 1)), Decimal(10.1))
+    checkEvaluation(cast(10.05, DecimalType(2, 0)), Decimal(10))
+    checkEvaluation(cast(10.05, DecimalType(1, 0)), null)
+    checkEvaluation(cast(10.05, DecimalType(2, 1)), null)
+    checkEvaluation(cast(10.05, DecimalType(3, 2)), null)
+    checkEvaluation(cast(Decimal(10.05), DecimalType(3, 1)), Decimal(10.1))
+    checkEvaluation(cast(Decimal(10.05), DecimalType(3, 2)), null)
+
+    checkEvaluation(cast(9.95, DecimalType(3, 2)), Decimal(9.95))
+    checkEvaluation(cast(9.95, DecimalType(3, 1)), Decimal(10.0))
+    checkEvaluation(cast(9.95, DecimalType(2, 0)), Decimal(10))
+    checkEvaluation(cast(9.95, DecimalType(2, 1)), null)
+    checkEvaluation(cast(9.95, DecimalType(1, 0)), null)
+    checkEvaluation(cast(Decimal(9.95), DecimalType(3, 1)), Decimal(10.0))
+    checkEvaluation(cast(Decimal(9.95), DecimalType(1, 0)), null)
+
+    checkEvaluation(cast(-9.95, DecimalType(3, 2)), Decimal(-9.95))
+    checkEvaluation(cast(-9.95, DecimalType(3, 1)), Decimal(-10.0))
+    checkEvaluation(cast(-9.95, DecimalType(2, 0)), Decimal(-10))
+    checkEvaluation(cast(-9.95, DecimalType(2, 1)), null)
+    checkEvaluation(cast(-9.95, DecimalType(1, 0)), null)
+    checkEvaluation(cast(Decimal(-9.95), DecimalType(3, 1)), Decimal(-10.0))
+    checkEvaluation(cast(Decimal(-9.95), DecimalType(1, 0)), null)
+
+    checkEvaluation(cast(Double.NaN, DecimalType.Unlimited), null)
+    checkEvaluation(cast(1.0 / 0.0, DecimalType.Unlimited), null)
+    checkEvaluation(cast(Float.NaN, DecimalType.Unlimited), null)
+    checkEvaluation(cast(1.0f / 0.0f, DecimalType.Unlimited), null)
+
+    checkEvaluation(cast(Double.NaN, DecimalType(2, 1)), null)
+    checkEvaluation(cast(1.0 / 0.0, DecimalType(2, 1)), null)
+    checkEvaluation(cast(Float.NaN, DecimalType(2, 1)), null)
+    checkEvaluation(cast(1.0f / 0.0f, DecimalType(2, 1)), null)
+  }
+
+  test("cast from date") {
+    val d = Date.valueOf("1970-01-01")
+    checkEvaluation(cast(d, ShortType), null)
+    checkEvaluation(cast(d, IntegerType), null)
+    checkEvaluation(cast(d, LongType), null)
+    checkEvaluation(cast(d, FloatType), null)
+    checkEvaluation(cast(d, DoubleType), null)
+    checkEvaluation(cast(d, DecimalType.Unlimited), null)
+    checkEvaluation(cast(d, DecimalType(10, 2)), null)
+    checkEvaluation(cast(d, StringType), "1970-01-01")
+    checkEvaluation(cast(cast(d, TimestampType), StringType), "1970-01-01 00:00:00")
+  }
+
+  test("cast from timestamp") {
+    val millis = 15 * 1000 + 2
+    val seconds = millis * 1000 + 2
+    val ts = new Timestamp(millis)
+    val tss = new Timestamp(seconds)
+    checkEvaluation(cast(ts, ShortType), 15.toShort)
+    checkEvaluation(cast(ts, IntegerType), 15)
+    checkEvaluation(cast(ts, LongType), 15.toLong)
+    checkEvaluation(cast(ts, FloatType), 15.002f)
+    checkEvaluation(cast(ts, DoubleType), 15.002)
+    checkEvaluation(cast(cast(tss, ShortType), TimestampType), ts)
+    checkEvaluation(cast(cast(tss, IntegerType), TimestampType), ts)
+    checkEvaluation(cast(cast(tss, LongType), TimestampType), ts)
+    checkEvaluation(
+      cast(cast(millis.toFloat / 1000, TimestampType), FloatType),
+      millis.toFloat / 1000)
+    checkEvaluation(
+      cast(cast(millis.toDouble / 1000, TimestampType), DoubleType),
+      millis.toDouble / 1000)
+    checkEvaluation(
+      cast(cast(Decimal(1), TimestampType), DecimalType.Unlimited),
+      Decimal(1))
+
+    // A test for higher precision than millis
+    checkEvaluation(cast(cast(0.00000001, TimestampType), DoubleType), 0.00000001)
+
+    checkEvaluation(cast(Double.NaN, TimestampType), null)
+    checkEvaluation(cast(1.0 / 0.0, TimestampType), null)
+    checkEvaluation(cast(Float.NaN, TimestampType), null)
+    checkEvaluation(cast(1.0f / 0.0f, TimestampType), null)
+  }
+
+  test("cast from array") {
+    val array = Literal.create(Seq("123", "abc", "", null),
+      ArrayType(StringType, containsNull = true))
+    val array_notNull = Literal.create(Seq("123", "abc", ""),
+      ArrayType(StringType, containsNull = false))
+
+    {
+      val ret = cast(array, ArrayType(IntegerType, containsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Seq(123, null, null, null))
+    }
+    {
+      val ret = cast(array, ArrayType(IntegerType, containsNull = false))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(array, ArrayType(BooleanType, containsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Seq(true, true, false, null))
+    }
+    {
+      val ret = cast(array, ArrayType(BooleanType, containsNull = false))
+      assert(ret.resolved === false)
+    }
+
+    {
+      val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Seq(123, null, null))
+    }
+    {
+      val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = false))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Seq(true, true, false))
+    }
+    {
+      val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = false))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Seq(true, true, false))
+    }
+
+    {
+      val ret = cast(array, IntegerType)
+      assert(ret.resolved === false)
+    }
+  }
+
+  test("cast from map") {
+    val map = Literal.create(
+      Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null),
+      MapType(StringType, StringType, valueContainsNull = true))
+    val map_notNull = Literal.create(
+      Map("a" -> "123", "b" -> "abc", "c" -> ""),
+      MapType(StringType, StringType, valueContainsNull = false))
+
+    {
+      val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null))
+    }
+    {
+      val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = false))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(map, MapType(StringType, BooleanType, valueContainsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Map("a" -> true, "b" -> true, "c" -> false, "d" -> null))
+    }
+    {
+      val ret = cast(map, MapType(StringType, BooleanType, valueContainsNull = false))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(map, MapType(IntegerType, StringType, valueContainsNull = true))
+      assert(ret.resolved === false)
+    }
+
+    {
+      val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null))
+    }
+    {
+      val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Map("a" -> true, "b" -> true, "c" -> false))
+    }
+    {
+      val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Map("a" -> true, "b" -> true, "c" -> false))
+    }
+    {
+      val ret = cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true))
+      assert(ret.resolved === false)
+    }
+
+    {
+      val ret = cast(map, IntegerType)
+      assert(ret.resolved === false)
+    }
+  }
+
+  test("cast from struct") {
+    val struct = Literal.create(
+      Row("123", "abc", "", null),
+      StructType(Seq(
+        StructField("a", StringType, nullable = true),
+        StructField("b", StringType, nullable = true),
+        StructField("c", StringType, nullable = true),
+        StructField("d", StringType, nullable = true))))
+    val struct_notNull = Literal.create(
+      Row("123", "abc", ""),
+      StructType(Seq(
+        StructField("a", StringType, nullable = false),
+        StructField("b", StringType, nullable = false),
+        StructField("c", StringType, nullable = false))))
+
+    {
+      val ret = cast(struct, StructType(Seq(
+        StructField("a", IntegerType, nullable = true),
+        StructField("b", IntegerType, nullable = true),
+        StructField("c", IntegerType, nullable = true),
+        StructField("d", IntegerType, nullable = true))))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Row(123, null, null, null))
+    }
+    {
+      val ret = cast(struct, StructType(Seq(
+        StructField("a", IntegerType, nullable = true),
+        StructField("b", IntegerType, nullable = true),
+        StructField("c", IntegerType, nullable = false),
+        StructField("d", IntegerType, nullable = true))))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(struct, StructType(Seq(
+        StructField("a", BooleanType, nullable = true),
+        StructField("b", BooleanType, nullable = true),
+        StructField("c", BooleanType, nullable = true),
+        StructField("d", BooleanType, nullable = true))))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Row(true, true, false, null))
+    }
+    {
+      val ret = cast(struct, StructType(Seq(
+        StructField("a", BooleanType, nullable = true),
+        StructField("b", BooleanType, nullable = true),
+        StructField("c", BooleanType, nullable = false),
+        StructField("d", BooleanType, nullable = true))))
+      assert(ret.resolved === false)
+    }
+
+    {
+      val ret = cast(struct_notNull, StructType(Seq(
+        StructField("a", IntegerType, nullable = true),
+        StructField("b", IntegerType, nullable = true),
+        StructField("c", IntegerType, nullable = true))))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Row(123, null, null))
+    }
+    {
+      val ret = cast(struct_notNull, StructType(Seq(
+        StructField("a", IntegerType, nullable = true),
+        StructField("b", IntegerType, nullable = true),
+        StructField("c", IntegerType, nullable = false))))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(struct_notNull, StructType(Seq(
+        StructField("a", BooleanType, nullable = true),
+        StructField("b", BooleanType, nullable = true),
+        StructField("c", BooleanType, nullable = true))))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Row(true, true, false))
+    }
+    {
+      val ret = cast(struct_notNull, StructType(Seq(
+        StructField("a", BooleanType, nullable = true),
+        StructField("b", BooleanType, nullable = true),
+        StructField("c", BooleanType, nullable = false))))
+      assert(ret.resolved === true)
+      checkEvaluation(ret, Row(true, true, false))
+    }
+
+    {
+      val ret = cast(struct, StructType(Seq(
+        StructField("a", StringType, nullable = true),
+        StructField("b", StringType, nullable = true),
+        StructField("c", StringType, nullable = true))))
+      assert(ret.resolved === false)
+    }
+    {
+      val ret = cast(struct, IntegerType)
+      assert(ret.resolved === false)
+    }
+  }
+
+  test("complex casting") {
+    val complex = Literal.create(
+      Row(
+        Seq("123", "abc", ""),
+        Map("a" -> "123", "b" -> "abc", "c" -> ""),
+        Row(0)),
+      StructType(Seq(
+        StructField("a",
+          ArrayType(StringType, containsNull = false), nullable = true),
+        StructField("m",
+          MapType(StringType, StringType, valueContainsNull = false), nullable = true),
+        StructField("s",
+          StructType(Seq(
+            StructField("i", IntegerType, nullable = true)))))))
+
+    val ret = cast(complex, StructType(Seq(
+      StructField("a",
+        ArrayType(IntegerType, containsNull = true), nullable = true),
+      StructField("m",
+        MapType(StringType, BooleanType, valueContainsNull = false), nullable = true),
+      StructField("s",
+        StructType(Seq(
+          StructField("l", LongType, nullable = true)))))))
+
+    assert(ret.resolved === true)
+    checkEvaluation(ret, Row(
+      Seq(123, null, null),
+      Map("a" -> true, "b" -> true, "c" -> false),
+      Row(0L)))
+  }
+
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
similarity index 94%
rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala
rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
index 371a73181dad7e8f160f28512618072e645bff7a..481b335d15dfdf9ebf3ffc66c8e9087f0c36a530 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedEvaluationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGenerationSuite.scala
@@ -17,13 +17,14 @@
 
 package org.apache.spark.sql.catalyst.expressions
 
+import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.catalyst.dsl.expressions._
 import org.apache.spark.sql.catalyst.expressions.codegen._
 
 /**
  * Additional tests for code generation.
  */
-class GeneratedEvaluationSuite extends ExpressionEvaluationSuite {
+class CodeGenerationSuite extends SparkFunSuite {
 
   test("multithreaded eval") {
     import scala.concurrent._
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f151dd2a47f785bf346873815a373c2b06571ebe
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala
@@ -0,0 +1,122 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types._
+
+
+class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  test("CreateStruct") {
+    val row = Row(1, 2, 3)
+    val c1 = 'a.int.at(0).as("a")
+    val c3 = 'c.int.at(2).as("c")
+    checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row)
+  }
+
+  test("complex type") {
+    val row = create_row(
+      "^Ba*n",                                // 0
+      null.asInstanceOf[UTF8String],          // 1
+      create_row("aa", "bb"),                 // 2
+      Map("aa"->"bb"),                        // 3
+      Seq("aa", "bb")                         // 4
+    )
+
+    val typeS = StructType(
+      StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil
+    )
+    val typeMap = MapType(StringType, StringType)
+    val typeArray = ArrayType(StringType)
+
+    checkEvaluation(GetMapValue(BoundReference(3, typeMap, true),
+      Literal("aa")), "bb", row)
+    checkEvaluation(GetMapValue(Literal.create(null, typeMap), Literal("aa")), null, row)
+    checkEvaluation(
+      GetMapValue(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row)
+    checkEvaluation(GetMapValue(BoundReference(3, typeMap, true),
+      Literal.create(null, StringType)), null, row)
+
+    checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true),
+      Literal(1)), "bb", row)
+    checkEvaluation(GetArrayItem(Literal.create(null, typeArray), Literal(1)), null, row)
+    checkEvaluation(
+      GetArrayItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row)
+    checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true),
+      Literal.create(null, IntegerType)), null, row)
+
+    def getStructField(expr: Expression, fieldName: String): ExtractValue = {
+      expr.dataType match {
+        case StructType(fields) =>
+          val field = fields.find(_.name == fieldName).get
+          GetStructField(expr, field, fields.indexOf(field))
+      }
+    }
+
+    def quickResolve(u: UnresolvedExtractValue): ExtractValue = {
+      ExtractValue(u.child, u.extraction, _ == _)
+    }
+
+    checkEvaluation(getStructField(BoundReference(2, typeS, nullable = true), "a"), "aa", row)
+    checkEvaluation(getStructField(Literal.create(null, typeS), "a"), null, row)
+
+    val typeS_notNullable = StructType(
+      StructField("a", StringType, nullable = false)
+        :: StructField("b", StringType, nullable = false) :: Nil
+    )
+
+    assert(getStructField(BoundReference(2, typeS, nullable = true), "a").nullable === true)
+    assert(getStructField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable
+      === false)
+
+    assert(getStructField(Literal.create(null, typeS), "a").nullable === true)
+    assert(getStructField(Literal.create(null, typeS_notNullable), "a").nullable === true)
+
+    checkEvaluation(quickResolve('c.map(typeMap).at(3).getItem("aa")), "bb", row)
+    checkEvaluation(quickResolve('c.array(typeArray.elementType).at(4).getItem(1)), "bb", row)
+    checkEvaluation(quickResolve('c.struct(typeS).at(2).getField("a")), "aa", row)
+  }
+
+  test("error message of ExtractValue") {
+    val structType = StructType(StructField("a", StringType, true) :: Nil)
+    val arrayStructType = ArrayType(structType)
+    val arrayType = ArrayType(StringType)
+    val otherType = StringType
+
+    def checkErrorMessage(
+      childDataType: DataType,
+      fieldDataType: DataType,
+      errorMesage: String): Unit = {
+      val e = intercept[org.apache.spark.sql.AnalysisException] {
+        ExtractValue(
+          Literal.create(null, childDataType),
+          Literal.create(null, fieldDataType),
+          _ == _)
+      }
+      assert(e.getMessage().contains(errorMesage))
+    }
+
+    checkErrorMessage(structType, IntegerType, "Field name should be String Literal")
+    checkErrorMessage(arrayStructType, BooleanType, "Field name should be String Literal")
+    checkErrorMessage(arrayType, StringType, "Array index should be integral type")
+    checkErrorMessage(otherType, StringType, "Can't extract value from")
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..152c4e4111244746fee230e3a129fb058224e3eb
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala
@@ -0,0 +1,96 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types.{IntegerType, BooleanType}
+
+
+class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  test("case when") {
+    val row = create_row(null, false, true, "a", "b", "c")
+    val c1 = 'a.boolean.at(0)
+    val c2 = 'a.boolean.at(1)
+    val c3 = 'a.boolean.at(2)
+    val c4 = 'a.string.at(3)
+    val c5 = 'a.string.at(4)
+    val c6 = 'a.string.at(5)
+
+    checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row)
+    checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row)
+    checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row)
+    checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row)
+    checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row)
+    checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row)
+
+    checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row)
+    checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row)
+    checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row)
+    checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row)
+
+    assert(CaseWhen(Seq(c2, c4, c6)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true)
+
+    val c4_notNull = 'a.boolean.notNull.at(3)
+    val c5_notNull = 'a.boolean.notNull.at(4)
+    val c6_notNull = 'a.boolean.notNull.at(5)
+
+    assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false)
+    assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true)
+
+    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false)
+    assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true)
+
+    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true)
+    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true)
+  }
+
+  test("case key when") {
+    val row = create_row(null, 1, 2, "a", "b", "c")
+    val c1 = 'a.int.at(0)
+    val c2 = 'a.int.at(1)
+    val c3 = 'a.int.at(2)
+    val c4 = 'a.string.at(3)
+    val c5 = 'a.string.at(4)
+    val c6 = 'a.string.at(5)
+
+    val literalNull = Literal.create(null, IntegerType)
+    val literalInt = Literal(1)
+    val literalString = Literal("a")
+
+    checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row)
+    checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row)
+    checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row)
+    checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row)
+    checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row)
+    checkEvaluation(CaseKeyWhen(c4, Seq(c6, c3, c5, c2, Literal(3))), 3, row)
+
+    checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row)
+    checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row)
+    checkEvaluation(CaseKeyWhen(c6, Seq(c5, c2, c4, c3)), null, row)
+    checkEvaluation(CaseKeyWhen(literalNull, Seq(c2, c5, c1, c6)), "c", row)
+  }
+
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
new file mode 100644
index 0000000000000000000000000000000000000000..87a92b87962f8c85fd77f7529adb484a4b6f701d
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala
@@ -0,0 +1,134 @@
+/*
+ * 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.scalactic.TripleEqualsSupport.Spread
+import org.scalatest.Matchers._
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.CatalystTypeConverters
+import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateProjection, GenerateMutableProjection}
+
+/**
+ * A few helper functions for expression evaluation testing. Mixin this trait to use them.
+ */
+trait ExpressionEvalHelper {
+  self: SparkFunSuite =>
+
+  protected def create_row(values: Any*): Row = {
+    new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray)
+  }
+
+  protected def checkEvaluation(
+      expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = {
+    checkEvaluationWithoutCodegen(expression, expected, inputRow)
+    checkEvaluationWithGeneratedMutableProjection(expression, expected, inputRow)
+    checkEvaluationWithGeneratedProjection(expression, expected, inputRow)
+  }
+
+  protected def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = {
+    expression.eval(inputRow)
+  }
+
+  protected def checkEvaluationWithoutCodegen(
+      expression: Expression,
+      expected: Any,
+      inputRow: Row = EmptyRow): Unit = {
+    val actual = try evaluate(expression, inputRow) catch {
+      case e: Exception => fail(s"Exception evaluating $expression", e)
+    }
+    if (actual != expected) {
+      val input = if (inputRow == EmptyRow) "" else s", input: $inputRow"
+      fail(s"Incorrect evaluation (codegen off): $expression, " +
+        s"actual: $actual, " +
+        s"expected: $expected$input")
+    }
+  }
+
+  protected def checkEvaluationWithGeneratedMutableProjection(
+      expression: Expression,
+      expected: Any,
+      inputRow: Row = EmptyRow): Unit = {
+
+    val plan = try {
+      GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)()
+    } catch {
+      case e: Throwable =>
+        val ctx = GenerateProjection.newCodeGenContext()
+        val evaluated = expression.gen(ctx)
+        fail(
+          s"""
+            |Code generation of $expression failed:
+            |${evaluated.code}
+            |$e
+          """.stripMargin)
+    }
+
+    val actual = plan(inputRow).apply(0)
+    if (actual != expected) {
+      val input = if (inputRow == EmptyRow) "" else s", input: $inputRow"
+      fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input")
+    }
+  }
+
+  protected def checkEvaluationWithGeneratedProjection(
+      expression: Expression,
+      expected: Any,
+      inputRow: Row = EmptyRow): Unit = {
+    val ctx = GenerateProjection.newCodeGenContext()
+    lazy val evaluated = expression.gen(ctx)
+
+    val plan = try {
+      GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)
+    } catch {
+      case e: Throwable =>
+        fail(
+          s"""
+            |Code generation of $expression failed:
+            |${evaluated.code}
+            |$e
+          """.stripMargin)
+    }
+
+    val actual = plan(inputRow)
+    val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected)))
+    if (actual.hashCode() != expectedRow.hashCode()) {
+      fail(
+        s"""
+          |Mismatched hashCodes for values: $actual, $expectedRow
+          |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()}
+          |Expressions: $expression
+          |Code: $evaluated
+        """.stripMargin)
+    }
+    if (actual != expectedRow) {
+      val input = if (inputRow == EmptyRow) "" else s", input: $inputRow"
+      fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input")
+    }
+  }
+
+  protected def checkDoubleEvaluation(
+      expression: Expression,
+      expected: Spread[Double],
+      inputRow: Row = EmptyRow): Unit = {
+    val actual = try evaluate(expression, inputRow) catch {
+      case e: Exception => fail(s"Exception evaluating $expression", e)
+    }
+    actual.asInstanceOf[Double] shouldBe expected
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
deleted file mode 100644
index eea2edc323eea57224520262da9c0fb1086af670..0000000000000000000000000000000000000000
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
+++ /dev/null
@@ -1,1461 +0,0 @@
-/*
- * 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 java.sql.{Date, Timestamp}
-
-import scala.collection.immutable.HashSet
-
-import org.scalactic.TripleEqualsSupport.Spread
-import org.scalatest.Matchers._
-
-import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.CatalystTypeConverters
-import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue
-import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateProjection, GenerateMutableProjection}
-import org.apache.spark.sql.catalyst.expressions.mathfuncs._
-import org.apache.spark.sql.catalyst.util.DateUtils
-import org.apache.spark.sql.types._
-
-
-class ExpressionEvaluationBaseSuite extends SparkFunSuite {
-
-  def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = {
-    checkEvaluationWithoutCodegen(expression, expected, inputRow)
-    checkEvaluationWithGeneratedMutableProjection(expression, expected, inputRow)
-    checkEvaluationWithGeneratedProjection(expression, expected, inputRow)
-  }
-
-  def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = {
-    expression.eval(inputRow)
-  }
-
-  def checkEvaluationWithoutCodegen(
-      expression: Expression,
-      expected: Any,
-      inputRow: Row = EmptyRow): Unit = {
-    val actual = try evaluate(expression, inputRow) catch {
-      case e: Exception => fail(s"Exception evaluating $expression", e)
-    }
-    if (actual != expected) {
-      val input = if (inputRow == EmptyRow) "" else s", input: $inputRow"
-      fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input")
-    }
-  }
-
-  def checkEvaluationWithGeneratedMutableProjection(
-      expression: Expression,
-      expected: Any,
-      inputRow: Row = EmptyRow): Unit = {
-
-    val plan = try {
-      GenerateMutableProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)()
-    } catch {
-      case e: Throwable =>
-        val ctx = GenerateProjection.newCodeGenContext()
-        val evaluated = expression.gen(ctx)
-        fail(
-          s"""
-            |Code generation of $expression failed:
-            |${evaluated.code}
-            |$e
-          """.stripMargin)
-    }
-
-    val actual = plan(inputRow).apply(0)
-    if (actual != expected) {
-      val input = if (inputRow == EmptyRow) "" else s", input: $inputRow"
-      fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input")
-    }
-  }
-
-  def checkEvaluationWithGeneratedProjection(
-      expression: Expression,
-      expected: Any,
-      inputRow: Row = EmptyRow): Unit = {
-    val ctx = GenerateProjection.newCodeGenContext()
-    lazy val evaluated = expression.gen(ctx)
-
-    val plan = try {
-      GenerateProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil)
-    } catch {
-      case e: Throwable =>
-        fail(
-          s"""
-            |Code generation of $expression failed:
-            |${evaluated.code}
-            |$e
-          """.stripMargin)
-    }
-
-    val actual = plan(inputRow)
-    val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected)))
-    if (actual.hashCode() != expectedRow.hashCode()) {
-      fail(
-        s"""
-          |Mismatched hashCodes for values: $actual, $expectedRow
-          |Hash Codes: ${actual.hashCode()} != ${expectedRow.hashCode()}
-          |Expressions: ${expression}
-          |Code: ${evaluated}
-        """.stripMargin)
-    }
-    if (actual != expectedRow) {
-      val input = if (inputRow == EmptyRow) "" else s", input: $inputRow"
-      fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input")
-    }
-  }
-
-  def checkDoubleEvaluation(
-      expression: Expression,
-      expected: Spread[Double],
-      inputRow: Row = EmptyRow): Unit = {
-    val actual = try evaluate(expression, inputRow) catch {
-      case e: Exception => fail(s"Exception evaluating $expression", e)
-    }
-    actual.asInstanceOf[Double] shouldBe expected
-  }
-}
-
-class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite {
-
-  def create_row(values: Any*): Row = {
-    new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray)
-  }
-
-  test("literals") {
-    checkEvaluation(Literal(1), 1)
-    checkEvaluation(Literal(true), true)
-    checkEvaluation(Literal(false), false)
-    checkEvaluation(Literal(0L), 0L)
-    List(0.0, -0.0, Double.NegativeInfinity, Double.PositiveInfinity).foreach {
-      d => {
-        checkEvaluation(Literal(d), d)
-        checkEvaluation(Literal(d.toFloat), d.toFloat)
-      }
-    }
-    checkEvaluation(Literal("test"), "test")
-    checkEvaluation(Literal.create(null, StringType), null)
-    checkEvaluation(Literal(1) + Literal(1), 2)
-  }
-
-  test("unary BitwiseNOT") {
-    checkEvaluation(BitwiseNot(1), -2)
-    assert(BitwiseNot(1).dataType === IntegerType)
-    assert(BitwiseNot(1).eval(EmptyRow).isInstanceOf[Int])
-    checkEvaluation(BitwiseNot(1.toLong), -2.toLong)
-    assert(BitwiseNot(1.toLong).dataType === LongType)
-    assert(BitwiseNot(1.toLong).eval(EmptyRow).isInstanceOf[Long])
-    checkEvaluation(BitwiseNot(1.toShort), -2.toShort)
-    assert(BitwiseNot(1.toShort).dataType === ShortType)
-    assert(BitwiseNot(1.toShort).eval(EmptyRow).isInstanceOf[Short])
-    checkEvaluation(BitwiseNot(1.toByte), -2.toByte)
-    assert(BitwiseNot(1.toByte).dataType === ByteType)
-    assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte])
-  }
-
-  // scalastyle:off
-  /**
-   * Checks for three-valued-logic.  Based on:
-   * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29
-   * I.e. in flat cpo "False -> Unknown -> True",
-   *   OR is lowest upper bound,
-   *   AND is greatest lower bound.
-   * p       q       p OR q  p AND q  p = q
-   * True    True    True    True     True
-   * True    False   True    False    False
-   * True    Unknown True    Unknown  Unknown
-   * False   True    True    False    False
-   * False   False   False   False    True
-   * False   Unknown Unknown False    Unknown
-   * Unknown True    True    Unknown  Unknown
-   * Unknown False   Unknown False    Unknown
-   * Unknown Unknown Unknown Unknown  Unknown
-   *
-   * p       NOT p
-   * True    False
-   * False   True
-   * Unknown Unknown
-   */
-  // scalastyle:on
-  val notTrueTable =
-    (true, false) ::
-    (false, true) ::
-    (null, null) :: Nil
-
-  test("3VL Not") {
-    notTrueTable.foreach {
-      case (v, answer) =>
-        checkEvaluation(!Literal.create(v, BooleanType), answer)
-    }
-  }
-
-  booleanLogicTest("AND", _ && _,
-    (true, true, true) ::
-    (true, false, false) ::
-    (true, null, null) ::
-    (false, true, false) ::
-    (false, false, false) ::
-    (false, null, false) ::
-    (null, true, null) ::
-    (null, false, false) ::
-    (null, null, null) :: Nil)
-
-  booleanLogicTest("OR", _ || _,
-    (true, true, true) ::
-    (true, false, true) ::
-    (true, null, true) ::
-    (false, true, true) ::
-    (false, false, false) ::
-    (false, null, null) ::
-    (null, true, true) ::
-    (null, false, null) ::
-    (null, null, null) :: Nil)
-
-  booleanLogicTest("=", _ === _,
-    (true, true, true) ::
-    (true, false, false) ::
-    (true, null, null) ::
-    (false, true, false) ::
-    (false, false, true) ::
-    (false, null, null) ::
-    (null, true, null) ::
-    (null, false, null) ::
-    (null, null, null) :: Nil)
-
-  def booleanLogicTest(
-      name: String,
-      op: (Expression, Expression) => Expression,
-      truthTable: Seq[(Any, Any, Any)]) {
-    test(s"3VL $name") {
-      truthTable.foreach {
-        case (l, r, answer) =>
-          val expr = op(Literal.create(l, BooleanType), Literal.create(r, BooleanType))
-          checkEvaluation(expr, answer)
-      }
-    }
-  }
-
-  test("IN") {
-    checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true)
-    checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true)
-    checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false)
-    checkEvaluation(
-      In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))),
-      true)
-  }
-
-  test("Divide") {
-    checkEvaluation(Divide(Literal(2), Literal(1)), 2)
-    checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5)
-    checkEvaluation(Divide(Literal(1), Literal(2)), 0)
-    checkEvaluation(Divide(Literal(1), Literal(0)), null)
-    checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null)
-    checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null)
-    checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null)
-    checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null)
-    checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null)
-    checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null)
-    checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null)
-    checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)),
-      null)
-  }
-
-  test("Remainder") {
-    checkEvaluation(Remainder(Literal(2), Literal(1)), 0)
-    checkEvaluation(Remainder(Literal(1.0), Literal(2.0)), 1.0)
-    checkEvaluation(Remainder(Literal(1), Literal(2)), 1)
-    checkEvaluation(Remainder(Literal(1), Literal(0)), null)
-    checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null)
-    checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null)
-    checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null)
-    checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null)
-    checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null)
-    checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null)
-    checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null)
-    checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)),
-      null)
-  }
-
-  test("INSET") {
-    val hS = HashSet[Any]() + 1 + 2
-    val nS = HashSet[Any]() + 1 + 2 + null
-    val one = Literal(1)
-    val two = Literal(2)
-    val three = Literal(3)
-    val nl = Literal(null)
-    val s = Seq(one, two)
-    val nullS = Seq(one, two, null)
-    checkEvaluation(InSet(one, hS), true)
-    checkEvaluation(InSet(two, hS), true)
-    checkEvaluation(InSet(two, nS), true)
-    checkEvaluation(InSet(nl, nS), true)
-    checkEvaluation(InSet(three, hS), false)
-    checkEvaluation(InSet(three, nS), false)
-    checkEvaluation(InSet(one, hS) && InSet(two, hS), true)
-  }
-
-  test("MaxOf") {
-    checkEvaluation(MaxOf(1, 2), 2)
-    checkEvaluation(MaxOf(2, 1), 2)
-    checkEvaluation(MaxOf(1L, 2L), 2L)
-    checkEvaluation(MaxOf(2L, 1L), 2L)
-
-    checkEvaluation(MaxOf(Literal.create(null, IntegerType), 2), 2)
-    checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2)
-  }
-
-  test("MinOf") {
-    checkEvaluation(MinOf(1, 2), 1)
-    checkEvaluation(MinOf(2, 1), 1)
-    checkEvaluation(MinOf(1L, 2L), 1L)
-    checkEvaluation(MinOf(2L, 1L), 1L)
-
-    checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1)
-    checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1)
-  }
-
-  test("LIKE literal Regular Expression") {
-    checkEvaluation(Literal.create(null, StringType).like("a"), null)
-    checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null)
-    checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null)
-    checkEvaluation("abdef" like "abdef", true)
-    checkEvaluation("a_%b" like "a\\__b", true)
-    checkEvaluation("addb" like "a_%b", true)
-    checkEvaluation("addb" like "a\\__b", false)
-    checkEvaluation("addb" like "a%\\%b", false)
-    checkEvaluation("a_%b" like "a%\\%b", true)
-    checkEvaluation("addb" like "a%", true)
-    checkEvaluation("addb" like "**", false)
-    checkEvaluation("abc" like "a%", true)
-    checkEvaluation("abc"  like "b%", false)
-    checkEvaluation("abc"  like "bc%", false)
-    checkEvaluation("a\nb" like "a_b", true)
-    checkEvaluation("ab" like "a%b", true)
-    checkEvaluation("a\nb" like "a%b", true)
-  }
-
-  test("LIKE Non-literal Regular Expression") {
-    val regEx = 'a.string.at(0)
-    checkEvaluation("abcd" like regEx, null, create_row(null))
-    checkEvaluation("abdef" like regEx, true, create_row("abdef"))
-    checkEvaluation("a_%b" like regEx, true, create_row("a\\__b"))
-    checkEvaluation("addb" like regEx, true, create_row("a_%b"))
-    checkEvaluation("addb" like regEx, false, create_row("a\\__b"))
-    checkEvaluation("addb" like regEx, false, create_row("a%\\%b"))
-    checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b"))
-    checkEvaluation("addb" like regEx, true, create_row("a%"))
-    checkEvaluation("addb" like regEx, false, create_row("**"))
-    checkEvaluation("abc" like regEx, true, create_row("a%"))
-    checkEvaluation("abc" like regEx, false, create_row("b%"))
-    checkEvaluation("abc" like regEx, false, create_row("bc%"))
-    checkEvaluation("a\nb" like regEx, true, create_row("a_b"))
-    checkEvaluation("ab" like regEx, true, create_row("a%b"))
-    checkEvaluation("a\nb" like regEx, true, create_row("a%b"))
-
-    checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%"))
-  }
-
-  test("RLIKE literal Regular Expression") {
-    checkEvaluation(Literal.create(null, StringType) rlike "abdef", null)
-    checkEvaluation("abdef" rlike Literal.create(null, StringType), null)
-    checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null)
-    checkEvaluation("abdef" rlike "abdef", true)
-    checkEvaluation("abbbbc" rlike "a.*c", true)
-
-    checkEvaluation("fofo" rlike "^fo", true)
-    checkEvaluation("fo\no" rlike "^fo\no$", true)
-    checkEvaluation("Bn" rlike "^Ba*n", true)
-    checkEvaluation("afofo" rlike "fo", true)
-    checkEvaluation("afofo" rlike "^fo", false)
-    checkEvaluation("Baan" rlike "^Ba?n", false)
-    checkEvaluation("axe" rlike "pi|apa", false)
-    checkEvaluation("pip" rlike "^(pi)*$", false)
-
-    checkEvaluation("abc"  rlike "^ab", true)
-    checkEvaluation("abc"  rlike "^bc", false)
-    checkEvaluation("abc"  rlike "^ab", true)
-    checkEvaluation("abc"  rlike "^bc", false)
-
-    intercept[java.util.regex.PatternSyntaxException] {
-      evaluate("abbbbc" rlike "**")
-    }
-  }
-
-  test("RLIKE Non-literal Regular Expression") {
-    val regEx = 'a.string.at(0)
-    checkEvaluation("abdef" rlike regEx, true, create_row("abdef"))
-    checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c"))
-    checkEvaluation("fofo" rlike regEx, true, create_row("^fo"))
-    checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$"))
-    checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n"))
-
-    intercept[java.util.regex.PatternSyntaxException] {
-      evaluate("abbbbc" rlike regEx, create_row("**"))
-    }
-  }
-
-  test("data type casting") {
-
-    val sd = "1970-01-01"
-    val d = Date.valueOf(sd)
-    val zts = sd + " 00:00:00"
-    val sts = sd + " 00:00:02"
-    val nts = sts + ".1"
-    val ts = Timestamp.valueOf(nts)
-
-    checkEvaluation("abdef" cast StringType, "abdef")
-    checkEvaluation("abdef" cast DecimalType.Unlimited, null)
-    checkEvaluation("abdef" cast TimestampType, null)
-    checkEvaluation("12.65" cast DecimalType.Unlimited, Decimal(12.65))
-
-    checkEvaluation(Literal(1) cast LongType, 1.toLong)
-    checkEvaluation(Cast(Literal(1000) cast TimestampType, LongType), 1.toLong)
-    checkEvaluation(Cast(Literal(-1200) cast TimestampType, LongType), -2.toLong)
-    checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble)
-    checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble)
-
-    checkEvaluation(Cast(Literal(sd) cast DateType, StringType), sd)
-    checkEvaluation(Cast(Literal(d) cast StringType, DateType), 0)
-    checkEvaluation(Cast(Literal(nts) cast TimestampType, StringType), nts)
-    checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts)
-    // all convert to string type to check
-    checkEvaluation(
-      Cast(Cast(Literal(nts) cast TimestampType, DateType), StringType), sd)
-    checkEvaluation(
-      Cast(Cast(Literal(ts) cast DateType, TimestampType), StringType), zts)
-
-    checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef")
-
-    checkEvaluation(Cast(Cast(Cast(Cast(
-      Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType),
-      5.toLong)
-    checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast
-      ByteType, TimestampType), DecimalType.Unlimited), LongType), StringType), ShortType),
-      0.toShort)
-    checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast
-      TimestampType, ByteType), DecimalType.Unlimited), LongType), StringType), ShortType), null)
-    checkEvaluation(Cast(Cast(Cast(Cast(Cast("5" cast
-      DecimalType.Unlimited, ByteType), TimestampType), LongType), StringType), ShortType),
-      0.toShort)
-    checkEvaluation(Literal(true) cast IntegerType, 1)
-    checkEvaluation(Literal(false) cast IntegerType, 0)
-    checkEvaluation(Literal(true) cast StringType, "true")
-    checkEvaluation(Literal(false) cast StringType, "false")
-    checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1)
-    checkEvaluation(Cast(Literal(0) cast BooleanType, IntegerType), 0)
-    checkEvaluation("23" cast DoubleType, 23d)
-    checkEvaluation("23" cast IntegerType, 23)
-    checkEvaluation("23" cast FloatType, 23f)
-    checkEvaluation("23" cast DecimalType.Unlimited, Decimal(23))
-    checkEvaluation("23" cast ByteType, 23.toByte)
-    checkEvaluation("23" cast ShortType, 23.toShort)
-    checkEvaluation("2012-12-11" cast DoubleType, null)
-    checkEvaluation(Literal(123) cast IntegerType, 123)
-
-    checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24d)
-    checkEvaluation(Literal(23) + Cast(true, IntegerType), 24)
-    checkEvaluation(Literal(23f) + Cast(true, FloatType), 24f)
-    checkEvaluation(Literal(Decimal(23)) + Cast(true, DecimalType.Unlimited), Decimal(24))
-    checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24.toByte)
-    checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24.toShort)
-
-    intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)}
-
-    assert(("abcdef" cast StringType).nullable === false)
-    assert(("abcdef" cast BinaryType).nullable === false)
-    assert(("abcdef" cast BooleanType).nullable === false)
-    assert(("abcdef" cast TimestampType).nullable === true)
-    assert(("abcdef" cast LongType).nullable === true)
-    assert(("abcdef" cast IntegerType).nullable === true)
-    assert(("abcdef" cast ShortType).nullable === true)
-    assert(("abcdef" cast ByteType).nullable === true)
-    assert(("abcdef" cast DecimalType.Unlimited).nullable === true)
-    assert(("abcdef" cast DecimalType(4, 2)).nullable === true)
-    assert(("abcdef" cast DoubleType).nullable === true)
-    assert(("abcdef" cast FloatType).nullable === true)
-
-    checkEvaluation(Cast(Literal.create(null, IntegerType), ShortType), null)
-  }
-
-  test("date") {
-    val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01"))
-    val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02"))
-    checkEvaluation(Literal(d1) < Literal(d2), true)
-  }
-
-  test("casting to fixed-precision decimals") {
-    // Overflow and rounding for casting to fixed-precision decimals:
-    // - Values should round with HALF_UP mode by default when you lower scale
-    // - Values that would overflow the target precision should turn into null
-    // - Because of this, casts to fixed-precision decimals should be nullable
-
-    assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false)
-    assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === true)
-    assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === true)
-    assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false)
-
-    assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true)
-    assert(Cast(Literal(10.03f), DecimalType(2, 1)).nullable === true)
-    assert(Cast(Literal(10.03), DecimalType(2, 1)).nullable === true)
-    assert(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable === true)
-
-    checkEvaluation(Cast(Literal(123), DecimalType.Unlimited), Decimal(123))
-    checkEvaluation(Cast(Literal(123), DecimalType(3, 0)), Decimal(123))
-    checkEvaluation(Cast(Literal(123), DecimalType(3, 1)), null)
-    checkEvaluation(Cast(Literal(123), DecimalType(2, 0)), null)
-
-    checkEvaluation(Cast(Literal(10.03), DecimalType.Unlimited), Decimal(10.03))
-    checkEvaluation(Cast(Literal(10.03), DecimalType(4, 2)), Decimal(10.03))
-    checkEvaluation(Cast(Literal(10.03), DecimalType(3, 1)), Decimal(10.0))
-    checkEvaluation(Cast(Literal(10.03), DecimalType(2, 0)), Decimal(10))
-    checkEvaluation(Cast(Literal(10.03), DecimalType(1, 0)), null)
-    checkEvaluation(Cast(Literal(10.03), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(10.03), DecimalType(3, 2)), null)
-    checkEvaluation(Cast(Literal(Decimal(10.03)), DecimalType(3, 1)), Decimal(10.0))
-    checkEvaluation(Cast(Literal(Decimal(10.03)), DecimalType(3, 2)), null)
-
-    checkEvaluation(Cast(Literal(10.05), DecimalType.Unlimited), Decimal(10.05))
-    checkEvaluation(Cast(Literal(10.05), DecimalType(4, 2)), Decimal(10.05))
-    checkEvaluation(Cast(Literal(10.05), DecimalType(3, 1)), Decimal(10.1))
-    checkEvaluation(Cast(Literal(10.05), DecimalType(2, 0)), Decimal(10))
-    checkEvaluation(Cast(Literal(10.05), DecimalType(1, 0)), null)
-    checkEvaluation(Cast(Literal(10.05), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(10.05), DecimalType(3, 2)), null)
-    checkEvaluation(Cast(Literal(Decimal(10.05)), DecimalType(3, 1)), Decimal(10.1))
-    checkEvaluation(Cast(Literal(Decimal(10.05)), DecimalType(3, 2)), null)
-
-    checkEvaluation(Cast(Literal(9.95), DecimalType(3, 2)), Decimal(9.95))
-    checkEvaluation(Cast(Literal(9.95), DecimalType(3, 1)), Decimal(10.0))
-    checkEvaluation(Cast(Literal(9.95), DecimalType(2, 0)), Decimal(10))
-    checkEvaluation(Cast(Literal(9.95), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(9.95), DecimalType(1, 0)), null)
-    checkEvaluation(Cast(Literal(Decimal(9.95)), DecimalType(3, 1)), Decimal(10.0))
-    checkEvaluation(Cast(Literal(Decimal(9.95)), DecimalType(1, 0)), null)
-
-    checkEvaluation(Cast(Literal(-9.95), DecimalType(3, 2)), Decimal(-9.95))
-    checkEvaluation(Cast(Literal(-9.95), DecimalType(3, 1)), Decimal(-10.0))
-    checkEvaluation(Cast(Literal(-9.95), DecimalType(2, 0)), Decimal(-10))
-    checkEvaluation(Cast(Literal(-9.95), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(-9.95), DecimalType(1, 0)), null)
-    checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(3, 1)), Decimal(-10.0))
-    checkEvaluation(Cast(Literal(Decimal(-9.95)), DecimalType(1, 0)), null)
-
-    checkEvaluation(Cast(Literal(Double.NaN), DecimalType.Unlimited), null)
-    checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType.Unlimited), null)
-    checkEvaluation(Cast(Literal(Float.NaN), DecimalType.Unlimited), null)
-    checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType.Unlimited), null)
-
-    checkEvaluation(Cast(Literal(Double.NaN), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(1.0 / 0.0), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(Float.NaN), DecimalType(2, 1)), null)
-    checkEvaluation(Cast(Literal(1.0f / 0.0f), DecimalType(2, 1)), null)
-  }
-
-  test("timestamp") {
-    val ts1 = new Timestamp(12)
-    val ts2 = new Timestamp(123)
-    checkEvaluation(Literal("ab") < Literal("abc"), true)
-    checkEvaluation(Literal(ts1) < Literal(ts2), true)
-  }
-
-  test("date casting") {
-    val d = Date.valueOf("1970-01-01")
-    checkEvaluation(Cast(Literal(d), ShortType), null)
-    checkEvaluation(Cast(Literal(d), IntegerType), null)
-    checkEvaluation(Cast(Literal(d), LongType), null)
-    checkEvaluation(Cast(Literal(d), FloatType), null)
-    checkEvaluation(Cast(Literal(d), DoubleType), null)
-    checkEvaluation(Cast(Literal(d), DecimalType.Unlimited), null)
-    checkEvaluation(Cast(Literal(d), DecimalType(10, 2)), null)
-    checkEvaluation(Cast(Literal(d), StringType), "1970-01-01")
-    checkEvaluation(Cast(Cast(Literal(d), TimestampType), StringType), "1970-01-01 00:00:00")
-  }
-
-  test("timestamp casting") {
-    val millis = 15 * 1000 + 2
-    val seconds = millis * 1000 + 2
-    val ts = new Timestamp(millis)
-    val tss = new Timestamp(seconds)
-    checkEvaluation(Cast(ts, ShortType), 15.toShort)
-    checkEvaluation(Cast(ts, IntegerType), 15)
-    checkEvaluation(Cast(ts, LongType), 15.toLong)
-    checkEvaluation(Cast(ts, FloatType), 15.002f)
-    checkEvaluation(Cast(ts, DoubleType), 15.002)
-    checkEvaluation(Cast(Cast(tss, ShortType), TimestampType), ts)
-    checkEvaluation(Cast(Cast(tss, IntegerType), TimestampType), ts)
-    checkEvaluation(Cast(Cast(tss, LongType), TimestampType), ts)
-    checkEvaluation(Cast(Cast(millis.toFloat / 1000, TimestampType), FloatType),
-      millis.toFloat / 1000)
-    checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType),
-      millis.toDouble / 1000)
-    checkEvaluation(Cast(Literal(Decimal(1)) cast TimestampType, DecimalType.Unlimited), Decimal(1))
-
-    // A test for higher precision than millis
-    checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001)
-
-    checkEvaluation(Cast(Literal(Double.NaN), TimestampType), null)
-    checkEvaluation(Cast(Literal(1.0 / 0.0), TimestampType), null)
-    checkEvaluation(Cast(Literal(Float.NaN), TimestampType), null)
-    checkEvaluation(Cast(Literal(1.0f / 0.0f), TimestampType), null)
-  }
-
-  test("array casting") {
-    val array = Literal.create(Seq("123", "abc", "", null),
-      ArrayType(StringType, containsNull = true))
-    val array_notNull = Literal.create(Seq("123", "abc", ""),
-      ArrayType(StringType, containsNull = false))
-
-    {
-      val cast = Cast(array, ArrayType(IntegerType, containsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Seq(123, null, null, null))
-    }
-    {
-      val cast = Cast(array, ArrayType(IntegerType, containsNull = false))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(array, ArrayType(BooleanType, containsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Seq(true, true, false, null))
-    }
-    {
-      val cast = Cast(array, ArrayType(BooleanType, containsNull = false))
-      assert(cast.resolved === false)
-    }
-
-    {
-      val cast = Cast(array_notNull, ArrayType(IntegerType, containsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Seq(123, null, null))
-    }
-    {
-      val cast = Cast(array_notNull, ArrayType(IntegerType, containsNull = false))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(array_notNull, ArrayType(BooleanType, containsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Seq(true, true, false))
-    }
-    {
-      val cast = Cast(array_notNull, ArrayType(BooleanType, containsNull = false))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Seq(true, true, false))
-    }
-
-    {
-      val cast = Cast(array, IntegerType)
-      assert(cast.resolved === false)
-    }
-  }
-
-  test("map casting") {
-    val map = Literal.create(
-      Map("a" -> "123", "b" -> "abc", "c" -> "", "d" -> null),
-      MapType(StringType, StringType, valueContainsNull = true))
-    val map_notNull = Literal.create(
-      Map("a" -> "123", "b" -> "abc", "c" -> ""),
-      MapType(StringType, StringType, valueContainsNull = false))
-
-    {
-      val cast = Cast(map, MapType(StringType, IntegerType, valueContainsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null))
-    }
-    {
-      val cast = Cast(map, MapType(StringType, IntegerType, valueContainsNull = false))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(map, MapType(StringType, BooleanType, valueContainsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false, "d" -> null))
-    }
-    {
-      val cast = Cast(map, MapType(StringType, BooleanType, valueContainsNull = false))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(map, MapType(IntegerType, StringType, valueContainsNull = true))
-      assert(cast.resolved === false)
-    }
-
-    {
-      val cast = Cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Map("a" -> 123, "b" -> null, "c" -> null))
-    }
-    {
-      val cast = Cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = false))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false))
-    }
-    {
-      val cast = Cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = false))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Map("a" -> true, "b" -> true, "c" -> false))
-    }
-    {
-      val cast = Cast(map_notNull, MapType(IntegerType, StringType, valueContainsNull = true))
-      assert(cast.resolved === false)
-    }
-
-    {
-      val cast = Cast(map, IntegerType)
-      assert(cast.resolved === false)
-    }
-  }
-
-  test("struct casting") {
-    val struct = Literal.create(
-      Row("123", "abc", "", null),
-      StructType(Seq(
-        StructField("a", StringType, nullable = true),
-        StructField("b", StringType, nullable = true),
-        StructField("c", StringType, nullable = true),
-        StructField("d", StringType, nullable = true))))
-    val struct_notNull = Literal.create(
-      Row("123", "abc", ""),
-      StructType(Seq(
-        StructField("a", StringType, nullable = false),
-        StructField("b", StringType, nullable = false),
-        StructField("c", StringType, nullable = false))))
-
-    {
-      val cast = Cast(struct, StructType(Seq(
-        StructField("a", IntegerType, nullable = true),
-        StructField("b", IntegerType, nullable = true),
-        StructField("c", IntegerType, nullable = true),
-        StructField("d", IntegerType, nullable = true))))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Row(123, null, null, null))
-    }
-    {
-      val cast = Cast(struct, StructType(Seq(
-        StructField("a", IntegerType, nullable = true),
-        StructField("b", IntegerType, nullable = true),
-        StructField("c", IntegerType, nullable = false),
-        StructField("d", IntegerType, nullable = true))))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(struct, StructType(Seq(
-        StructField("a", BooleanType, nullable = true),
-        StructField("b", BooleanType, nullable = true),
-        StructField("c", BooleanType, nullable = true),
-        StructField("d", BooleanType, nullable = true))))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Row(true, true, false, null))
-    }
-    {
-      val cast = Cast(struct, StructType(Seq(
-        StructField("a", BooleanType, nullable = true),
-        StructField("b", BooleanType, nullable = true),
-        StructField("c", BooleanType, nullable = false),
-        StructField("d", BooleanType, nullable = true))))
-      assert(cast.resolved === false)
-    }
-
-    {
-      val cast = Cast(struct_notNull, StructType(Seq(
-        StructField("a", IntegerType, nullable = true),
-        StructField("b", IntegerType, nullable = true),
-        StructField("c", IntegerType, nullable = true))))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Row(123, null, null))
-    }
-    {
-      val cast = Cast(struct_notNull, StructType(Seq(
-        StructField("a", IntegerType, nullable = true),
-        StructField("b", IntegerType, nullable = true),
-        StructField("c", IntegerType, nullable = false))))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(struct_notNull, StructType(Seq(
-        StructField("a", BooleanType, nullable = true),
-        StructField("b", BooleanType, nullable = true),
-        StructField("c", BooleanType, nullable = true))))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Row(true, true, false))
-    }
-    {
-      val cast = Cast(struct_notNull, StructType(Seq(
-        StructField("a", BooleanType, nullable = true),
-        StructField("b", BooleanType, nullable = true),
-        StructField("c", BooleanType, nullable = false))))
-      assert(cast.resolved === true)
-      checkEvaluation(cast, Row(true, true, false))
-    }
-
-    {
-      val cast = Cast(struct, StructType(Seq(
-        StructField("a", StringType, nullable = true),
-        StructField("b", StringType, nullable = true),
-        StructField("c", StringType, nullable = true))))
-      assert(cast.resolved === false)
-    }
-    {
-      val cast = Cast(struct, IntegerType)
-      assert(cast.resolved === false)
-    }
-  }
-
-  test("complex casting") {
-    val complex = Literal.create(
-      Row(
-        Seq("123", "abc", ""),
-        Map("a" -> "123", "b" -> "abc", "c" -> ""),
-        Row(0)),
-      StructType(Seq(
-        StructField("a",
-          ArrayType(StringType, containsNull = false), nullable = true),
-        StructField("m",
-          MapType(StringType, StringType, valueContainsNull = false), nullable = true),
-        StructField("s",
-          StructType(Seq(
-            StructField("i", IntegerType, nullable = true)))))))
-
-    val cast = Cast(complex, StructType(Seq(
-      StructField("a",
-        ArrayType(IntegerType, containsNull = true), nullable = true),
-      StructField("m",
-        MapType(StringType, BooleanType, valueContainsNull = false), nullable = true),
-      StructField("s",
-        StructType(Seq(
-          StructField("l", LongType, nullable = true)))))))
-
-    assert(cast.resolved === true)
-    checkEvaluation(cast, Row(
-      Seq(123, null, null),
-      Map("a" -> true, "b" -> true, "c" -> false),
-      Row(0L)))
-  }
-
-  test("null checking") {
-    val row = create_row("^Ba*n", null, true, null)
-    val c1 = 'a.string.at(0)
-    val c2 = 'a.string.at(1)
-    val c3 = 'a.boolean.at(2)
-    val c4 = 'a.boolean.at(3)
-
-    checkEvaluation(c1.isNull, false, row)
-    checkEvaluation(c1.isNotNull, true, row)
-
-    checkEvaluation(c2.isNull, true, row)
-    checkEvaluation(c2.isNotNull, false, row)
-
-    checkEvaluation(Literal.create(1, ShortType).isNull, false)
-    checkEvaluation(Literal.create(1, ShortType).isNotNull, true)
-
-    checkEvaluation(Literal.create(null, ShortType).isNull, true)
-    checkEvaluation(Literal.create(null, ShortType).isNotNull, false)
-
-    checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row)
-    checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row)
-    checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row)
-
-    checkEvaluation(
-      If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row)
-    checkEvaluation(If(c3, c1, c2), "^Ba*n", row)
-    checkEvaluation(If(c4, c2, c1), "^Ba*n", row)
-    checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row)
-    checkEvaluation(If(Literal.create(true, BooleanType), c1, c2), "^Ba*n", row)
-    checkEvaluation(If(Literal.create(false, BooleanType), c2, c1), "^Ba*n", row)
-    checkEvaluation(If(Literal.create(false, BooleanType),
-      Literal.create("a", StringType), Literal.create("b", StringType)), "b", row)
-
-    checkEvaluation(c1 in (c1, c2), true, row)
-    checkEvaluation(
-      Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType)), true, row)
-    checkEvaluation(
-      Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType), c2), true, row)
-  }
-
-  test("case when") {
-    val row = create_row(null, false, true, "a", "b", "c")
-    val c1 = 'a.boolean.at(0)
-    val c2 = 'a.boolean.at(1)
-    val c3 = 'a.boolean.at(2)
-    val c4 = 'a.string.at(3)
-    val c5 = 'a.string.at(4)
-    val c6 = 'a.string.at(5)
-
-    checkEvaluation(CaseWhen(Seq(c1, c4, c6)), "c", row)
-    checkEvaluation(CaseWhen(Seq(c2, c4, c6)), "c", row)
-    checkEvaluation(CaseWhen(Seq(c3, c4, c6)), "a", row)
-    checkEvaluation(CaseWhen(Seq(Literal.create(null, BooleanType), c4, c6)), "c", row)
-    checkEvaluation(CaseWhen(Seq(Literal.create(false, BooleanType), c4, c6)), "c", row)
-    checkEvaluation(CaseWhen(Seq(Literal.create(true, BooleanType), c4, c6)), "a", row)
-
-    checkEvaluation(CaseWhen(Seq(c3, c4, c2, c5, c6)), "a", row)
-    checkEvaluation(CaseWhen(Seq(c2, c4, c3, c5, c6)), "b", row)
-    checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row)
-    checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row)
-
-    assert(CaseWhen(Seq(c2, c4, c6)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true)
-
-    val c4_notNull = 'a.boolean.notNull.at(3)
-    val c5_notNull = 'a.boolean.notNull.at(4)
-    val c6_notNull = 'a.boolean.notNull.at(5)
-
-    assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false)
-    assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true)
-
-    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false)
-    assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true)
-
-    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true)
-    assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true)
-  }
-
-  test("case key when") {
-    val row = create_row(null, 1, 2, "a", "b", "c")
-    val c1 = 'a.int.at(0)
-    val c2 = 'a.int.at(1)
-    val c3 = 'a.int.at(2)
-    val c4 = 'a.string.at(3)
-    val c5 = 'a.string.at(4)
-    val c6 = 'a.string.at(5)
-
-    val literalNull = Literal.create(null, IntegerType)
-    val literalInt = Literal(1)
-    val literalString = Literal("a")
-
-    checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, c5)), "b", row)
-    checkEvaluation(CaseKeyWhen(c1, Seq(c2, c4, literalNull, c5, c6)), "b", row)
-    checkEvaluation(CaseKeyWhen(c2, Seq(literalInt, c4, c5)), "a", row)
-    checkEvaluation(CaseKeyWhen(c2, Seq(c1, c4, c5)), "b", row)
-    checkEvaluation(CaseKeyWhen(c4, Seq(literalString, c2, c3)), 1, row)
-    checkEvaluation(CaseKeyWhen(c4, Seq(c6, c3, c5, c2, Literal(3))), 3, row)
-
-    checkEvaluation(CaseKeyWhen(literalInt, Seq(c2, c4, c5)), "a", row)
-    checkEvaluation(CaseKeyWhen(literalString, Seq(c5, c2, c4, c3)), 2, row)
-    checkEvaluation(CaseKeyWhen(c6, Seq(c5, c2, c4, c3)), null, row)
-    checkEvaluation(CaseKeyWhen(literalNull, Seq(c2, c5, c1, c6)), "c", row)
-  }
-
-  test("complex type") {
-    val row = create_row(
-      "^Ba*n",                                // 0
-      null.asInstanceOf[UTF8String],          // 1
-      create_row("aa", "bb"),                 // 2
-      Map("aa"->"bb"),                        // 3
-      Seq("aa", "bb")                         // 4
-    )
-
-    val typeS = StructType(
-      StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil
-    )
-    val typeMap = MapType(StringType, StringType)
-    val typeArray = ArrayType(StringType)
-
-    checkEvaluation(GetMapValue(BoundReference(3, typeMap, true),
-      Literal("aa")), "bb", row)
-    checkEvaluation(GetMapValue(Literal.create(null, typeMap), Literal("aa")), null, row)
-    checkEvaluation(
-      GetMapValue(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row)
-    checkEvaluation(GetMapValue(BoundReference(3, typeMap, true),
-      Literal.create(null, StringType)), null, row)
-
-    checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true),
-      Literal(1)), "bb", row)
-    checkEvaluation(GetArrayItem(Literal.create(null, typeArray), Literal(1)), null, row)
-    checkEvaluation(
-      GetArrayItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row)
-    checkEvaluation(GetArrayItem(BoundReference(4, typeArray, true),
-      Literal.create(null, IntegerType)), null, row)
-
-    def getStructField(expr: Expression, fieldName: String): ExtractValue = {
-      expr.dataType match {
-        case StructType(fields) =>
-          val field = fields.find(_.name == fieldName).get
-          GetStructField(expr, field, fields.indexOf(field))
-      }
-    }
-
-    def quickResolve(u: UnresolvedExtractValue): ExtractValue = {
-      ExtractValue(u.child, u.extraction, _ == _)
-    }
-
-    checkEvaluation(getStructField(BoundReference(2, typeS, nullable = true), "a"), "aa", row)
-    checkEvaluation(getStructField(Literal.create(null, typeS), "a"), null, row)
-
-    val typeS_notNullable = StructType(
-      StructField("a", StringType, nullable = false)
-        :: StructField("b", StringType, nullable = false) :: Nil
-    )
-
-    assert(getStructField(BoundReference(2, typeS, nullable = true), "a").nullable === true)
-    assert(getStructField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable
-      === false)
-
-    assert(getStructField(Literal.create(null, typeS), "a").nullable === true)
-    assert(getStructField(Literal.create(null, typeS_notNullable), "a").nullable === true)
-
-    checkEvaluation(quickResolve('c.map(typeMap).at(3).getItem("aa")), "bb", row)
-    checkEvaluation(quickResolve('c.array(typeArray.elementType).at(4).getItem(1)), "bb", row)
-    checkEvaluation(quickResolve('c.struct(typeS).at(2).getField("a")), "aa", row)
-  }
-
-  test("error message of ExtractValue") {
-    val structType = StructType(StructField("a", StringType, true) :: Nil)
-    val arrayStructType = ArrayType(structType)
-    val arrayType = ArrayType(StringType)
-    val otherType = StringType
-
-    def checkErrorMessage(
-        childDataType: DataType,
-        fieldDataType: DataType,
-        errorMesage: String): Unit = {
-      val e = intercept[org.apache.spark.sql.AnalysisException] {
-        ExtractValue(
-          Literal.create(null, childDataType),
-          Literal.create(null, fieldDataType),
-          _ == _)
-      }
-      assert(e.getMessage().contains(errorMesage))
-    }
-
-    checkErrorMessage(structType, IntegerType, "Field name should be String Literal")
-    checkErrorMessage(arrayStructType, BooleanType, "Field name should be String Literal")
-    checkErrorMessage(arrayType, StringType, "Array index should be integral type")
-    checkErrorMessage(otherType, StringType, "Can't extract value from")
-  }
-
-  test("arithmetic") {
-    val row = create_row(1, 2, 3, null)
-    val c1 = 'a.int.at(0)
-    val c2 = 'a.int.at(1)
-    val c3 = 'a.int.at(2)
-    val c4 = 'a.int.at(3)
-
-    checkEvaluation(UnaryMinus(c1), -1, row)
-    checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100)
-
-    checkEvaluation(Add(c1, c4), null, row)
-    checkEvaluation(Add(c1, c2), 3, row)
-    checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row)
-    checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row)
-    checkEvaluation(
-      Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
-
-    checkEvaluation(-c1, -1, row)
-    checkEvaluation(c1 + c2, 3, row)
-    checkEvaluation(c1 - c2, -1, row)
-    checkEvaluation(c1 * c2, 2, row)
-    checkEvaluation(c1 / c2, 0, row)
-    checkEvaluation(c1 % c2, 1, row)
-  }
-
-  test("fractional arithmetic") {
-    val row = create_row(1.1, 2.0, 3.1, null)
-    val c1 = 'a.double.at(0)
-    val c2 = 'a.double.at(1)
-    val c3 = 'a.double.at(2)
-    val c4 = 'a.double.at(3)
-
-    checkEvaluation(UnaryMinus(c1), -1.1, row)
-    checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0)
-    checkEvaluation(Add(c1, c4), null, row)
-    checkEvaluation(Add(c1, c2), 3.1, row)
-    checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row)
-    checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row)
-    checkEvaluation(
-      Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row)
-
-    checkEvaluation(-c1, -1.1, row)
-    checkEvaluation(c1 + c2, 3.1, row)
-    checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row)
-    checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row)
-    checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row)
-    checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row)
-  }
-
-  test("BinaryComparison") {
-    val row = create_row(1, 2, 3, null, 3, null)
-    val c1 = 'a.int.at(0)
-    val c2 = 'a.int.at(1)
-    val c3 = 'a.int.at(2)
-    val c4 = 'a.int.at(3)
-    val c5 = 'a.int.at(4)
-    val c6 = 'a.int.at(5)
-
-    checkEvaluation(LessThan(c1, c4), null, row)
-    checkEvaluation(LessThan(c1, c2), true, row)
-    checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row)
-    checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row)
-    checkEvaluation(
-      LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
-
-    checkEvaluation(c1 < c2, true, row)
-    checkEvaluation(c1 <= c2, true, row)
-    checkEvaluation(c1 > c2, false, row)
-    checkEvaluation(c1 >= c2, false, row)
-    checkEvaluation(c1 === c2, false, row)
-    checkEvaluation(c1 !== c2, true, row)
-    checkEvaluation(c4 <=> c1, false, row)
-    checkEvaluation(c1 <=> c4, false, row)
-    checkEvaluation(c4 <=> c6, true, row)
-    checkEvaluation(c3 <=> c5, true, row)
-    checkEvaluation(Literal(true) <=> Literal.create(null, BooleanType), false, row)
-    checkEvaluation(Literal.create(null, BooleanType) <=> Literal(true), false, row)
-  }
-
-  test("StringComparison") {
-    val row = create_row("abc", null)
-    val c1 = 'a.string.at(0)
-    val c2 = 'a.string.at(1)
-
-    checkEvaluation(c1 contains "b", true, row)
-    checkEvaluation(c1 contains "x", false, row)
-    checkEvaluation(c2 contains "b", null, row)
-    checkEvaluation(c1 contains Literal.create(null, StringType), null, row)
-
-    checkEvaluation(c1 startsWith "a", true, row)
-    checkEvaluation(c1 startsWith "b", false, row)
-    checkEvaluation(c2 startsWith "a", null, row)
-    checkEvaluation(c1 startsWith Literal.create(null, StringType), null, row)
-
-    checkEvaluation(c1 endsWith "c", true, row)
-    checkEvaluation(c1 endsWith "b", false, row)
-    checkEvaluation(c2 endsWith "b", null, row)
-    checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row)
-  }
-
-  test("Substring") {
-    val row = create_row("example", "example".toArray.map(_.toByte))
-
-    val s = 'a.string.at(0)
-
-    // substring from zero position with less-than-full length
-    checkEvaluation(
-      Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row)
-    checkEvaluation(
-      Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row)
-
-    // substring from zero position with full length
-    checkEvaluation(
-      Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row)
-    checkEvaluation(
-      Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row)
-
-    // substring from zero position with greater-than-full length
-    checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)),
-      "example", row)
-    checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)),
-      "example", row)
-
-    // substring from nonzero position with less-than-full length
-    checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)),
-      "xa", row)
-
-    // substring from nonzero position with full length
-    checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)),
-      "xample", row)
-
-    // substring from nonzero position with greater-than-full length
-    checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)),
-      "xample", row)
-
-    // zero-length substring (within string bounds)
-    checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)),
-      "", row)
-
-    // zero-length substring (beyond string bounds)
-    checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)),
-      "", row)
-
-    // substring(null, _, _) -> null
-    checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)),
-      null, create_row(null))
-
-    // substring(_, null, _) -> null
-    checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)),
-      null, row)
-
-    // substring(_, _, null) -> null
-    checkEvaluation(
-      Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)),
-      null,
-      row)
-
-    // 2-arg substring from zero position
-    checkEvaluation(
-      Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)),
-      "example",
-      row)
-    checkEvaluation(
-      Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)),
-      "example",
-      row)
-
-    // 2-arg substring from nonzero position
-    checkEvaluation(
-      Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)),
-      "xample",
-      row)
-
-    val s_notNull = 'a.string.notNull.at(0)
-
-    assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable
-      === true)
-    assert(
-      Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable
-        === false)
-    assert(Substring(s_notNull,
-      Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true)
-    assert(Substring(s_notNull,
-      Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true)
-
-    checkEvaluation(s.substr(0, 2), "ex", row)
-    checkEvaluation(s.substr(0), "example", row)
-    checkEvaluation(s.substring(0, 2), "ex", row)
-    checkEvaluation(s.substring(0), "example", row)
-  }
-
-  test("SQRT") {
-    val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24))
-    val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble))
-    val rowSequence = inputSequence.map(l => create_row(l.toDouble))
-    val d = 'a.double.at(0)
-
-    for ((row, expected) <- rowSequence zip expectedResults) {
-      checkEvaluation(Sqrt(d), expected, row)
-    }
-
-    checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null))
-    checkEvaluation(Sqrt(-1), null, EmptyRow)
-    checkEvaluation(Sqrt(-1.5), null, EmptyRow)
-  }
-
-  test("Bitwise operations") {
-    val row = create_row(1, 2, 3, null)
-    val c1 = 'a.int.at(0)
-    val c2 = 'a.int.at(1)
-    val c3 = 'a.int.at(2)
-    val c4 = 'a.int.at(3)
-
-    checkEvaluation(BitwiseAnd(c1, c4), null, row)
-    checkEvaluation(BitwiseAnd(c1, c2), 0, row)
-    checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row)
-    checkEvaluation(
-      BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
-
-    checkEvaluation(BitwiseOr(c1, c4), null, row)
-    checkEvaluation(BitwiseOr(c1, c2), 3, row)
-    checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row)
-    checkEvaluation(
-      BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
-
-    checkEvaluation(BitwiseXor(c1, c4), null, row)
-    checkEvaluation(BitwiseXor(c1, c2), 3, row)
-    checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row)
-    checkEvaluation(
-      BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
-
-    checkEvaluation(BitwiseNot(c4), null, row)
-    checkEvaluation(BitwiseNot(c1), -2, row)
-    checkEvaluation(BitwiseNot(Literal.create(null, IntegerType)), null, row)
-
-    checkEvaluation(c1 & c2, 0, row)
-    checkEvaluation(c1 | c2, 3, row)
-    checkEvaluation(c1 ^ c2, 3, row)
-    checkEvaluation(~c1, -2, row)
-  }
-
-  /**
-   * Used for testing math functions for DataFrames.
-   * @param c The DataFrame function
-   * @param f The functions in scala.math
-   * @param domain The set of values to run the function with
-   * @param expectNull Whether the given values should return null or not
-   * @tparam T Generic type for primitives
-   */
-  def unaryMathFunctionEvaluation[@specialized(Int, Double, Float, Long) T](
-      c: Expression => Expression,
-      f: T => T,
-      domain: Iterable[T] = (-20 to 20).map(_ * 0.1),
-      expectNull: Boolean = false): Unit = {
-    if (expectNull) {
-      domain.foreach { value =>
-        checkEvaluation(c(Literal(value)), null, EmptyRow)
-      }
-    } else {
-      domain.foreach { value =>
-        checkEvaluation(c(Literal(value)), f(value), EmptyRow)
-      }
-    }
-    checkEvaluation(c(Literal.create(null, DoubleType)), null, create_row(null))
-  }
-
-  test("sin") {
-    unaryMathFunctionEvaluation(Sin, math.sin)
-  }
-
-  test("asin") {
-    unaryMathFunctionEvaluation(Asin, math.asin, (-10 to 10).map(_ * 0.1))
-    unaryMathFunctionEvaluation(Asin, math.asin, (11 to 20).map(_ * 0.1), true)
-  }
-
-  test("sinh") {
-    unaryMathFunctionEvaluation(Sinh, math.sinh)
-  }
-
-  test("cos") {
-    unaryMathFunctionEvaluation(Cos, math.cos)
-  }
-
-  test("acos") {
-    unaryMathFunctionEvaluation(Acos, math.acos, (-10 to 10).map(_ * 0.1))
-    unaryMathFunctionEvaluation(Acos, math.acos, (11 to 20).map(_ * 0.1), true)
-  }
-
-  test("cosh") {
-    unaryMathFunctionEvaluation(Cosh, math.cosh)
-  }
-
-  test("tan") {
-    unaryMathFunctionEvaluation(Tan, math.tan)
-  }
-
-  test("atan") {
-    unaryMathFunctionEvaluation(Atan, math.atan)
-  }
-
-  test("tanh") {
-    unaryMathFunctionEvaluation(Tanh, math.tanh)
-  }
-
-  test("toDegrees") {
-    unaryMathFunctionEvaluation(ToDegrees, math.toDegrees)
-  }
-
-  test("toRadians") {
-    unaryMathFunctionEvaluation(ToRadians, math.toRadians)
-  }
-
-  test("cbrt") {
-    unaryMathFunctionEvaluation(Cbrt, math.cbrt)
-  }
-
-  test("ceil") {
-    unaryMathFunctionEvaluation(Ceil, math.ceil)
-  }
-
-  test("floor") {
-    unaryMathFunctionEvaluation(Floor, math.floor)
-  }
-
-  test("rint") {
-    unaryMathFunctionEvaluation(Rint, math.rint)
-  }
-
-  test("exp") {
-    unaryMathFunctionEvaluation(Exp, math.exp)
-  }
-
-  test("expm1") {
-    unaryMathFunctionEvaluation(Expm1, math.expm1)
-  }
-
-  test("signum") {
-    unaryMathFunctionEvaluation[Double](Signum, math.signum)
-  }
-
-  test("log") {
-    unaryMathFunctionEvaluation(Log, math.log, (0 to 20).map(_ * 0.1))
-    unaryMathFunctionEvaluation(Log, math.log, (-5 to -1).map(_ * 0.1), true)
-  }
-
-  test("log10") {
-    unaryMathFunctionEvaluation(Log10, math.log10, (0 to 20).map(_ * 0.1))
-    unaryMathFunctionEvaluation(Log10, math.log10, (-5 to -1).map(_ * 0.1), true)
-  }
-
-  test("log1p") {
-    unaryMathFunctionEvaluation(Log1p, math.log1p, (-1 to 20).map(_ * 0.1))
-    unaryMathFunctionEvaluation(Log1p, math.log1p, (-10 to -2).map(_ * 1.0), true)
-  }
-
-  /**
-   * Used for testing math functions for DataFrames.
-   * @param c The DataFrame function
-   * @param f The functions in scala.math
-   * @param domain The set of values to run the function with
-   */
-  def binaryMathFunctionEvaluation(
-      c: (Expression, Expression) => Expression,
-      f: (Double, Double) => Double,
-      domain: Iterable[(Double, Double)] = (-20 to 20).map(v => (v * 0.1, v * -0.1)),
-      expectNull: Boolean = false): Unit = {
-    if (expectNull) {
-      domain.foreach { case (v1, v2) =>
-        checkEvaluation(c(v1, v2), null, create_row(null))
-      }
-    } else {
-      domain.foreach { case (v1, v2) =>
-        checkEvaluation(c(v1, v2), f(v1 + 0.0, v2 + 0.0), EmptyRow)
-        checkEvaluation(c(v2, v1), f(v2 + 0.0, v1 + 0.0), EmptyRow)
-      }
-    }
-    checkEvaluation(c(Literal.create(null, DoubleType), 1.0), null, create_row(null))
-    checkEvaluation(c(1.0, Literal.create(null, DoubleType)), null, create_row(null))
-  }
-
-  test("pow") {
-    binaryMathFunctionEvaluation(Pow, math.pow, (-5 to 5).map(v => (v * 1.0, v * 1.0)))
-    binaryMathFunctionEvaluation(Pow, math.pow, Seq((-1.0, 0.9), (-2.2, 1.7), (-2.2, -1.7)), true)
-  }
-
-  test("hypot") {
-    binaryMathFunctionEvaluation(Hypot, math.hypot)
-  }
-
-  test("atan2") {
-    binaryMathFunctionEvaluation(Atan2, math.atan2)
-  }
-}
-
-// TODO: Make the tests work with codegen.
-class ExpressionEvaluationWithoutCodeGenSuite extends ExpressionEvaluationBaseSuite {
-
-  override def checkEvaluation(
-      expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = {
-    checkEvaluationWithoutCodegen(expression, expected, inputRow)
-  }
-
-  test("CreateStruct") {
-    val row = Row(1, 2, 3)
-    val c1 = 'a.int.at(0).as("a")
-    val c3 = 'c.int.at(2).as("c")
-    checkEvaluation(CreateStruct(Seq(c1, c3)), Row(1, 3), row)
-  }
-}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..f44f55dfb92d1e8908032bbab86882bed73ace1f
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala
@@ -0,0 +1,55 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.types.StringType
+
+
+class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  // TODO: Add tests for all data types.
+
+  test("boolean literals") {
+    checkEvaluation(Literal(true), true)
+    checkEvaluation(Literal(false), false)
+  }
+
+  test("int literals") {
+    checkEvaluation(Literal(1), 1)
+    checkEvaluation(Literal(0L), 0L)
+  }
+
+  test("double literals") {
+    List(0.0, -0.0, Double.NegativeInfinity, Double.PositiveInfinity).foreach {
+      d => {
+        checkEvaluation(Literal(d), d)
+        checkEvaluation(Literal(d.toFloat), d.toFloat)
+      }
+    }
+  }
+
+  test("string literals") {
+    checkEvaluation(Literal("test"), "test")
+    checkEvaluation(Literal.create(null, StringType), null)
+  }
+
+  test("sum two literals") {
+    checkEvaluation(Add(Literal(1), Literal(1)), 2)
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..25ebc70d095d868266860f2342459e4fa0e0fff5
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala
@@ -0,0 +1,179 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.types.DoubleType
+
+class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  /**
+   * Used for testing unary math expressions.
+   *
+   * @param c expression
+   * @param f The functions in scala.math
+   * @param domain The set of values to run the function with
+   * @param expectNull Whether the given values should return null or not
+   * @tparam T Generic type for primitives
+   */
+  private def testUnary[T](
+      c: Expression => Expression,
+      f: T => T,
+      domain: Iterable[T] = (-20 to 20).map(_ * 0.1),
+      expectNull: Boolean = false): Unit = {
+    if (expectNull) {
+      domain.foreach { value =>
+        checkEvaluation(c(Literal(value)), null, EmptyRow)
+      }
+    } else {
+      domain.foreach { value =>
+        checkEvaluation(c(Literal(value)), f(value), EmptyRow)
+      }
+    }
+    checkEvaluation(c(Literal.create(null, DoubleType)), null, create_row(null))
+  }
+
+  /**
+   * Used for testing binary math expressions.
+   *
+   * @param c The DataFrame function
+   * @param f The functions in scala.math
+   * @param domain The set of values to run the function with
+   */
+  private def testBinary(
+      c: (Expression, Expression) => Expression,
+      f: (Double, Double) => Double,
+      domain: Iterable[(Double, Double)] = (-20 to 20).map(v => (v * 0.1, v * -0.1)),
+      expectNull: Boolean = false): Unit = {
+    if (expectNull) {
+      domain.foreach { case (v1, v2) =>
+        checkEvaluation(c(Literal(v1), Literal(v2)), null, create_row(null))
+      }
+    } else {
+      domain.foreach { case (v1, v2) =>
+        checkEvaluation(c(Literal(v1), Literal(v2)), f(v1 + 0.0, v2 + 0.0), EmptyRow)
+        checkEvaluation(c(Literal(v2), Literal(v1)), f(v2 + 0.0, v1 + 0.0), EmptyRow)
+      }
+    }
+    checkEvaluation(c(Literal.create(null, DoubleType), Literal(1.0)), null, create_row(null))
+    checkEvaluation(c(Literal(1.0), Literal.create(null, DoubleType)), null, create_row(null))
+  }
+
+  test("sin") {
+    testUnary(Sin, math.sin)
+  }
+
+  test("asin") {
+    testUnary(Asin, math.asin, (-10 to 10).map(_ * 0.1))
+    testUnary(Asin, math.asin, (11 to 20).map(_ * 0.1), expectNull = true)
+  }
+
+  test("sinh") {
+    testUnary(Sinh, math.sinh)
+  }
+
+  test("cos") {
+    testUnary(Cos, math.cos)
+  }
+
+  test("acos") {
+    testUnary(Acos, math.acos, (-10 to 10).map(_ * 0.1))
+    testUnary(Acos, math.acos, (11 to 20).map(_ * 0.1), expectNull = true)
+  }
+
+  test("cosh") {
+    testUnary(Cosh, math.cosh)
+  }
+
+  test("tan") {
+    testUnary(Tan, math.tan)
+  }
+
+  test("atan") {
+    testUnary(Atan, math.atan)
+  }
+
+  test("tanh") {
+    testUnary(Tanh, math.tanh)
+  }
+
+  test("toDegrees") {
+    testUnary(ToDegrees, math.toDegrees)
+  }
+
+  test("toRadians") {
+    testUnary(ToRadians, math.toRadians)
+  }
+
+  test("cbrt") {
+    testUnary(Cbrt, math.cbrt)
+  }
+
+  test("ceil") {
+    testUnary(Ceil, math.ceil)
+  }
+
+  test("floor") {
+    testUnary(Floor, math.floor)
+  }
+
+  test("rint") {
+    testUnary(Rint, math.rint)
+  }
+
+  test("exp") {
+    testUnary(Exp, math.exp)
+  }
+
+  test("expm1") {
+    testUnary(Expm1, math.expm1)
+  }
+
+  test("signum") {
+    testUnary[Double](Signum, math.signum)
+  }
+
+  test("log") {
+    testUnary(Log, math.log, (0 to 20).map(_ * 0.1))
+    testUnary(Log, math.log, (-5 to -1).map(_ * 0.1), expectNull = true)
+  }
+
+  test("log10") {
+    testUnary(Log10, math.log10, (0 to 20).map(_ * 0.1))
+    testUnary(Log10, math.log10, (-5 to -1).map(_ * 0.1), expectNull = true)
+  }
+
+  test("log1p") {
+    testUnary(Log1p, math.log1p, (-1 to 20).map(_ * 0.1))
+    testUnary(Log1p, math.log1p, (-10 to -2).map(_ * 1.0), expectNull = true)
+  }
+
+  test("pow") {
+    testBinary(Pow, math.pow, (-5 to 5).map(v => (v * 1.0, v * 1.0)))
+    testBinary(Pow, math.pow, Seq((-1.0, 0.9), (-2.2, 1.7), (-2.2, -1.7)), expectNull = true)
+  }
+
+  test("hypot") {
+    testBinary(Hypot, math.hypot)
+  }
+
+  test("atan2") {
+    testBinary(Atan2, math.atan2)
+  }
+
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..ccdada8b56f837419da13f0cc3e9e727fd0d7c59
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullFunctionsSuite.scala
@@ -0,0 +1,65 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types.{BooleanType, StringType, ShortType}
+
+class NullFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  test("null checking") {
+    val row = create_row("^Ba*n", null, true, null)
+    val c1 = 'a.string.at(0)
+    val c2 = 'a.string.at(1)
+    val c3 = 'a.boolean.at(2)
+    val c4 = 'a.boolean.at(3)
+
+    checkEvaluation(c1.isNull, false, row)
+    checkEvaluation(c1.isNotNull, true, row)
+
+    checkEvaluation(c2.isNull, true, row)
+    checkEvaluation(c2.isNotNull, false, row)
+
+    checkEvaluation(Literal.create(1, ShortType).isNull, false)
+    checkEvaluation(Literal.create(1, ShortType).isNotNull, true)
+
+    checkEvaluation(Literal.create(null, ShortType).isNull, true)
+    checkEvaluation(Literal.create(null, ShortType).isNotNull, false)
+
+    checkEvaluation(Coalesce(c1 :: c2 :: Nil), "^Ba*n", row)
+    checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row)
+    checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row)
+
+    checkEvaluation(
+      If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row)
+    checkEvaluation(If(c3, c1, c2), "^Ba*n", row)
+    checkEvaluation(If(c4, c2, c1), "^Ba*n", row)
+    checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row)
+    checkEvaluation(If(Literal.create(true, BooleanType), c1, c2), "^Ba*n", row)
+    checkEvaluation(If(Literal.create(false, BooleanType), c2, c1), "^Ba*n", row)
+    checkEvaluation(If(Literal.create(false, BooleanType),
+      Literal.create("a", StringType), Literal.create("b", StringType)), "b", row)
+
+    checkEvaluation(c1 in (c1, c2), true, row)
+    checkEvaluation(
+      Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType)), true, row)
+    checkEvaluation(
+      Literal.create("^Ba*n", StringType) in (Literal.create("^Ba*n", StringType), c2), true, row)
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..b6261bfba07867aeba924a14fe8952e9ded00ab7
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala
@@ -0,0 +1,179 @@
+/*
+ * 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 java.sql.{Date, Timestamp}
+
+import scala.collection.immutable.HashSet
+
+import org.apache.spark.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.util.DateUtils
+import org.apache.spark.sql.types.{IntegerType, BooleanType}
+
+
+class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  private def booleanLogicTest(
+    name: String,
+    op: (Expression, Expression) => Expression,
+    truthTable: Seq[(Any, Any, Any)]) {
+    test(s"3VL $name") {
+      truthTable.foreach {
+        case (l, r, answer) =>
+          val expr = op(Literal.create(l, BooleanType), Literal.create(r, BooleanType))
+          checkEvaluation(expr, answer)
+      }
+    }
+  }
+
+  // scalastyle:off
+  /**
+   * Checks for three-valued-logic.  Based on:
+   * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29
+   * I.e. in flat cpo "False -> Unknown -> True",
+   *   OR is lowest upper bound,
+   *   AND is greatest lower bound.
+   * p       q       p OR q  p AND q  p = q
+   * True    True    True    True     True
+   * True    False   True    False    False
+   * True    Unknown True    Unknown  Unknown
+   * False   True    True    False    False
+   * False   False   False   False    True
+   * False   Unknown Unknown False    Unknown
+   * Unknown True    True    Unknown  Unknown
+   * Unknown False   Unknown False    Unknown
+   * Unknown Unknown Unknown Unknown  Unknown
+   *
+   * p       NOT p
+   * True    False
+   * False   True
+   * Unknown Unknown
+   */
+  // scalastyle:on
+  val notTrueTable =
+    (true, false) ::
+      (false, true) ::
+      (null, null) :: Nil
+
+  test("3VL Not") {
+    notTrueTable.foreach { case (v, answer) =>
+      checkEvaluation(Not(Literal.create(v, BooleanType)), answer)
+    }
+  }
+
+  booleanLogicTest("AND", And,
+    (true, true, true) ::
+      (true, false, false) ::
+      (true, null, null) ::
+      (false, true, false) ::
+      (false, false, false) ::
+      (false, null, false) ::
+      (null, true, null) ::
+      (null, false, false) ::
+      (null, null, null) :: Nil)
+
+  booleanLogicTest("OR", Or,
+    (true, true, true) ::
+      (true, false, true) ::
+      (true, null, true) ::
+      (false, true, true) ::
+      (false, false, false) ::
+      (false, null, null) ::
+      (null, true, true) ::
+      (null, false, null) ::
+      (null, null, null) :: Nil)
+
+  booleanLogicTest("=", EqualTo,
+    (true, true, true) ::
+      (true, false, false) ::
+      (true, null, null) ::
+      (false, true, false) ::
+      (false, false, true) ::
+      (false, null, null) ::
+      (null, true, null) ::
+      (null, false, null) ::
+      (null, null, null) :: Nil)
+
+  test("IN") {
+    checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true)
+    checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true)
+    checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false)
+    checkEvaluation(
+      And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), Literal(2)))),
+      true)
+  }
+
+  test("INSET") {
+    val hS = HashSet[Any]() + 1 + 2
+    val nS = HashSet[Any]() + 1 + 2 + null
+    val one = Literal(1)
+    val two = Literal(2)
+    val three = Literal(3)
+    val nl = Literal(null)
+    val s = Seq(one, two)
+    val nullS = Seq(one, two, null)
+    checkEvaluation(InSet(one, hS), true)
+    checkEvaluation(InSet(two, hS), true)
+    checkEvaluation(InSet(two, nS), true)
+    checkEvaluation(InSet(nl, nS), true)
+    checkEvaluation(InSet(three, hS), false)
+    checkEvaluation(InSet(three, nS), false)
+    checkEvaluation(And(InSet(one, hS), InSet(two, hS)), true)
+  }
+
+
+  test("BinaryComparison") {
+    val row = create_row(1, 2, 3, null, 3, null)
+    val c1 = 'a.int.at(0)
+    val c2 = 'a.int.at(1)
+    val c3 = 'a.int.at(2)
+    val c4 = 'a.int.at(3)
+    val c5 = 'a.int.at(4)
+    val c6 = 'a.int.at(5)
+
+    checkEvaluation(LessThan(c1, c4), null, row)
+    checkEvaluation(LessThan(c1, c2), true, row)
+    checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row)
+    checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row)
+    checkEvaluation(
+      LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row)
+
+    checkEvaluation(c1 < c2, true, row)
+    checkEvaluation(c1 <= c2, true, row)
+    checkEvaluation(c1 > c2, false, row)
+    checkEvaluation(c1 >= c2, false, row)
+    checkEvaluation(c1 === c2, false, row)
+    checkEvaluation(c1 !== c2, true, row)
+    checkEvaluation(c4 <=> c1, false, row)
+    checkEvaluation(c1 <=> c4, false, row)
+    checkEvaluation(c4 <=> c6, true, row)
+    checkEvaluation(c3 <=> c5, true, row)
+    checkEvaluation(Literal(true) <=> Literal.create(null, BooleanType), false, row)
+    checkEvaluation(Literal.create(null, BooleanType) <=> Literal(true), false, row)
+
+    val d1 = DateUtils.fromJavaDate(Date.valueOf("1970-01-01"))
+    val d2 = DateUtils.fromJavaDate(Date.valueOf("1970-01-02"))
+    checkEvaluation(Literal(d1) < Literal(d2), true)
+
+    val ts1 = new Timestamp(12)
+    val ts2 = new Timestamp(123)
+    checkEvaluation(Literal("ab") < Literal("abc"), true)
+    checkEvaluation(Literal(ts1) < Literal(ts2), true)
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala
new file mode 100644
index 0000000000000000000000000000000000000000..2e81296c4e623a3d4b7b3e684f2b82a45a1a0256
--- /dev/null
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala
@@ -0,0 +1,218 @@
+/*
+ * 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.SparkFunSuite
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.types.{IntegerType, StringType}
+
+
+class StringFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper {
+
+  test("StringComparison") {
+    val row = create_row("abc", null)
+    val c1 = 'a.string.at(0)
+    val c2 = 'a.string.at(1)
+
+    checkEvaluation(c1 contains "b", true, row)
+    checkEvaluation(c1 contains "x", false, row)
+    checkEvaluation(c2 contains "b", null, row)
+    checkEvaluation(c1 contains Literal.create(null, StringType), null, row)
+
+    checkEvaluation(c1 startsWith "a", true, row)
+    checkEvaluation(c1 startsWith "b", false, row)
+    checkEvaluation(c2 startsWith "a", null, row)
+    checkEvaluation(c1 startsWith Literal.create(null, StringType), null, row)
+
+    checkEvaluation(c1 endsWith "c", true, row)
+    checkEvaluation(c1 endsWith "b", false, row)
+    checkEvaluation(c2 endsWith "b", null, row)
+    checkEvaluation(c1 endsWith Literal.create(null, StringType), null, row)
+  }
+
+  test("Substring") {
+    val row = create_row("example", "example".toArray.map(_.toByte))
+
+    val s = 'a.string.at(0)
+
+    // substring from zero position with less-than-full length
+    checkEvaluation(
+      Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row)
+    checkEvaluation(
+      Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row)
+
+    // substring from zero position with full length
+    checkEvaluation(
+      Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row)
+    checkEvaluation(
+      Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row)
+
+    // substring from zero position with greater-than-full length
+    checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)),
+      "example", row)
+    checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)),
+      "example", row)
+
+    // substring from nonzero position with less-than-full length
+    checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)),
+      "xa", row)
+
+    // substring from nonzero position with full length
+    checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)),
+      "xample", row)
+
+    // substring from nonzero position with greater-than-full length
+    checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)),
+      "xample", row)
+
+    // zero-length substring (within string bounds)
+    checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)),
+      "", row)
+
+    // zero-length substring (beyond string bounds)
+    checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)),
+      "", row)
+
+    // substring(null, _, _) -> null
+    checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)),
+      null, create_row(null))
+
+    // substring(_, null, _) -> null
+    checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)),
+      null, row)
+
+    // substring(_, _, null) -> null
+    checkEvaluation(
+      Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)),
+      null,
+      row)
+
+    // 2-arg substring from zero position
+    checkEvaluation(
+      Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)),
+      "example",
+      row)
+    checkEvaluation(
+      Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)),
+      "example",
+      row)
+
+    // 2-arg substring from nonzero position
+    checkEvaluation(
+      Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)),
+      "xample",
+      row)
+
+    val s_notNull = 'a.string.notNull.at(0)
+
+    assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable
+      === true)
+    assert(
+      Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable
+        === false)
+    assert(Substring(s_notNull,
+      Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true)
+    assert(Substring(s_notNull,
+      Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true)
+
+    checkEvaluation(s.substr(0, 2), "ex", row)
+    checkEvaluation(s.substr(0), "example", row)
+    checkEvaluation(s.substring(0, 2), "ex", row)
+    checkEvaluation(s.substring(0), "example", row)
+  }
+
+  test("LIKE literal Regular Expression") {
+    checkEvaluation(Literal.create(null, StringType).like("a"), null)
+    checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null)
+    checkEvaluation(Literal.create(null, StringType).like(Literal.create(null, StringType)), null)
+    checkEvaluation("abdef" like "abdef", true)
+    checkEvaluation("a_%b" like "a\\__b", true)
+    checkEvaluation("addb" like "a_%b", true)
+    checkEvaluation("addb" like "a\\__b", false)
+    checkEvaluation("addb" like "a%\\%b", false)
+    checkEvaluation("a_%b" like "a%\\%b", true)
+    checkEvaluation("addb" like "a%", true)
+    checkEvaluation("addb" like "**", false)
+    checkEvaluation("abc" like "a%", true)
+    checkEvaluation("abc"  like "b%", false)
+    checkEvaluation("abc"  like "bc%", false)
+    checkEvaluation("a\nb" like "a_b", true)
+    checkEvaluation("ab" like "a%b", true)
+    checkEvaluation("a\nb" like "a%b", true)
+  }
+
+  test("LIKE Non-literal Regular Expression") {
+    val regEx = 'a.string.at(0)
+    checkEvaluation("abcd" like regEx, null, create_row(null))
+    checkEvaluation("abdef" like regEx, true, create_row("abdef"))
+    checkEvaluation("a_%b" like regEx, true, create_row("a\\__b"))
+    checkEvaluation("addb" like regEx, true, create_row("a_%b"))
+    checkEvaluation("addb" like regEx, false, create_row("a\\__b"))
+    checkEvaluation("addb" like regEx, false, create_row("a%\\%b"))
+    checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b"))
+    checkEvaluation("addb" like regEx, true, create_row("a%"))
+    checkEvaluation("addb" like regEx, false, create_row("**"))
+    checkEvaluation("abc" like regEx, true, create_row("a%"))
+    checkEvaluation("abc" like regEx, false, create_row("b%"))
+    checkEvaluation("abc" like regEx, false, create_row("bc%"))
+    checkEvaluation("a\nb" like regEx, true, create_row("a_b"))
+    checkEvaluation("ab" like regEx, true, create_row("a%b"))
+    checkEvaluation("a\nb" like regEx, true, create_row("a%b"))
+
+    checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%"))
+  }
+
+  test("RLIKE literal Regular Expression") {
+    checkEvaluation(Literal.create(null, StringType) rlike "abdef", null)
+    checkEvaluation("abdef" rlike Literal.create(null, StringType), null)
+    checkEvaluation(Literal.create(null, StringType) rlike Literal.create(null, StringType), null)
+    checkEvaluation("abdef" rlike "abdef", true)
+    checkEvaluation("abbbbc" rlike "a.*c", true)
+
+    checkEvaluation("fofo" rlike "^fo", true)
+    checkEvaluation("fo\no" rlike "^fo\no$", true)
+    checkEvaluation("Bn" rlike "^Ba*n", true)
+    checkEvaluation("afofo" rlike "fo", true)
+    checkEvaluation("afofo" rlike "^fo", false)
+    checkEvaluation("Baan" rlike "^Ba?n", false)
+    checkEvaluation("axe" rlike "pi|apa", false)
+    checkEvaluation("pip" rlike "^(pi)*$", false)
+
+    checkEvaluation("abc"  rlike "^ab", true)
+    checkEvaluation("abc"  rlike "^bc", false)
+    checkEvaluation("abc"  rlike "^ab", true)
+    checkEvaluation("abc"  rlike "^bc", false)
+
+    intercept[java.util.regex.PatternSyntaxException] {
+      evaluate("abbbbc" rlike "**")
+    }
+  }
+
+  test("RLIKE Non-literal Regular Expression") {
+    val regEx = 'a.string.at(0)
+    checkEvaluation("abdef" rlike regEx, true, create_row("abdef"))
+    checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c"))
+    checkEvaluation("fofo" rlike regEx, true, create_row("^fo"))
+    checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$"))
+    checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n"))
+
+    intercept[java.util.regex.PatternSyntaxException] {
+      evaluate("abbbbc" rlike regEx, create_row("**"))
+    }
+  }
+}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala
index a4a3a66b8b2292746863f100842cdb8329e69033..f33a18d53b1a96733de2e4a3fe37ec05fcd6404f 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ExpressionOptimizationSuite.scala
@@ -17,6 +17,7 @@
 
 package org.apache.spark.sql.catalyst.optimizer
 
+import org.apache.spark.SparkFunSuite
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 
@@ -24,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
  * Overrides our expression evaluation tests and reruns them after optimization has occured.  This
  * is to ensure that constant folding and other optimizations do not break anything.
  */
-class ExpressionOptimizationSuite extends ExpressionEvaluationSuite {
+class ExpressionOptimizationSuite extends SparkFunSuite with ExpressionEvalHelper {
   override def checkEvaluation(
       expression: Expression,
       expected: Any,
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
index 77327f2b84eaa79b930c2ff51df0bc3e6c80d0f4..454af47913bf1d3c9f565857d9acf353bcfc1a6d 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
@@ -24,7 +24,6 @@ import org.apache.spark.annotation.Experimental
 import org.apache.spark.sql.catalyst.ScalaReflection
 import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star}
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.mathfuncs._
 import org.apache.spark.sql.types._
 import org.apache.spark.util.Utils