You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by da...@apache.org on 2015/07/30 22:21:56 UTC

spark git commit: [SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation

Repository: spark
Updated Branches:
  refs/heads/master d8cfd531c -> 1abf7dc16


[SPARK-8186] [SPARK-8187] [SPARK-8194] [SPARK-8198] [SPARK-9133] [SPARK-9290] [SQL] functions: date_add, date_sub, add_months, months_between, time-interval calculation

This PR is based on #7589 , thanks to adrian-wang

Added SQL function date_add, date_sub, add_months, month_between, also add a rule for
add/subtract of date/timestamp and interval.

Closes #7589

cc rxin

Author: Daoyuan Wang <da...@intel.com>
Author: Davies Liu <da...@databricks.com>

Closes #7754 from davies/date_add and squashes the following commits:

e8c633a [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
9e8e085 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
6224ce4 [Davies Liu] fix conclict
bd18cd4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into date_add
e47ff2c [Davies Liu] add python api, fix date functions
01943d0 [Davies Liu] Merge branch 'master' into date_add
522e91a [Daoyuan Wang] fix
e8a639a [Daoyuan Wang] fix
42df486 [Daoyuan Wang] fix style
87c4b77 [Daoyuan Wang] function add_months, months_between and some fixes
1a68e03 [Daoyuan Wang] poc of time interval calculation
c506661 [Daoyuan Wang] function date_add , date_sub


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

Branch: refs/heads/master
Commit: 1abf7dc16ca1ba1777fe874c8b81fe6f2b0a6de5
Parents: d8cfd53
Author: Daoyuan Wang <da...@intel.com>
Authored: Thu Jul 30 13:21:46 2015 -0700
Committer: Davies Liu <da...@gmail.com>
Committed: Thu Jul 30 13:21:46 2015 -0700

----------------------------------------------------------------------
 python/pyspark/sql/functions.py                 |  76 +++++--
 .../catalyst/analysis/FunctionRegistry.scala    |   4 +
 .../catalyst/analysis/HiveTypeCoercion.scala    |  22 ++
 .../expressions/datetimeFunctions.scala         | 155 +++++++++++++-
 .../spark/sql/catalyst/util/DateTimeUtils.scala | 139 +++++++++++++
 .../analysis/HiveTypeCoercionSuite.scala        |  30 +++
 .../expressions/DateExpressionsSuite.scala      | 176 ++++++++++------
 .../sql/catalyst/util/DateTimeUtilsSuite.scala  | 205 +++++++++++--------
 .../scala/org/apache/spark/sql/functions.scala  |  29 +++
 .../apache/spark/sql/DateFunctionsSuite.scala   | 117 +++++++++++
 10 files changed, 791 insertions(+), 162 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/python/pyspark/sql/functions.py
----------------------------------------------------------------------
diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py
index d930f7d..a7295e2 100644
--- a/python/pyspark/sql/functions.py
+++ b/python/pyspark/sql/functions.py
@@ -59,7 +59,7 @@ __all__ = [
 __all__ += ['lag', 'lead', 'ntile']
 
 __all__ += [
-    'date_format',
+    'date_format', 'date_add', 'date_sub', 'add_months', 'months_between',
     'year', 'quarter', 'month', 'hour', 'minute', 'second',
     'dayofmonth', 'dayofyear', 'weekofyear']
 
@@ -716,7 +716,7 @@ def date_format(dateCol, format):
     [Row(date=u'04/08/2015')]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.date_format(dateCol, format))
+    return Column(sc._jvm.functions.date_format(_to_java_column(dateCol), format))
 
 
 @since(1.5)
@@ -729,7 +729,7 @@ def year(col):
     [Row(year=2015)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.year(col))
+    return Column(sc._jvm.functions.year(_to_java_column(col)))
 
 
 @since(1.5)
@@ -742,7 +742,7 @@ def quarter(col):
     [Row(quarter=2)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.quarter(col))
+    return Column(sc._jvm.functions.quarter(_to_java_column(col)))
 
 
 @since(1.5)
@@ -755,7 +755,7 @@ def month(col):
     [Row(month=4)]
    """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.month(col))
+    return Column(sc._jvm.functions.month(_to_java_column(col)))
 
 
 @since(1.5)
@@ -768,7 +768,7 @@ def dayofmonth(col):
     [Row(day=8)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.dayofmonth(col))
+    return Column(sc._jvm.functions.dayofmonth(_to_java_column(col)))
 
 
 @since(1.5)
@@ -781,7 +781,7 @@ def dayofyear(col):
     [Row(day=98)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.dayofyear(col))
+    return Column(sc._jvm.functions.dayofyear(_to_java_column(col)))
 
 
 @since(1.5)
@@ -794,7 +794,7 @@ def hour(col):
     [Row(hour=13)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.hour(col))
+    return Column(sc._jvm.functions.hour(_to_java_column(col)))
 
 
 @since(1.5)
@@ -807,7 +807,7 @@ def minute(col):
     [Row(minute=8)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.minute(col))
+    return Column(sc._jvm.functions.minute(_to_java_column(col)))
 
 
 @since(1.5)
@@ -820,7 +820,7 @@ def second(col):
     [Row(second=15)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.second(col))
+    return Column(sc._jvm.functions.second(_to_java_column(col)))
 
 
 @since(1.5)
@@ -829,11 +829,63 @@ def weekofyear(col):
     Extract the week number of a given date as integer.
 
     >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['a'])
-    >>> df.select(weekofyear('a').alias('week')).collect()
+    >>> df.select(weekofyear(df.a).alias('week')).collect()
     [Row(week=15)]
     """
     sc = SparkContext._active_spark_context
-    return Column(sc._jvm.functions.weekofyear(col))
+    return Column(sc._jvm.functions.weekofyear(_to_java_column(col)))
+
+
+@since(1.5)
+def date_add(start, days):
+    """
+    Returns the date that is `days` days after `start`
+
+    >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d'])
+    >>> df.select(date_add(df.d, 1).alias('d')).collect()
+    [Row(d=datetime.date(2015, 4, 9))]
+    """
+    sc = SparkContext._active_spark_context
+    return Column(sc._jvm.functions.date_add(_to_java_column(start), days))
+
+
+@since(1.5)
+def date_sub(start, days):
+    """
+    Returns the date that is `days` days before `start`
+
+    >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d'])
+    >>> df.select(date_sub(df.d, 1).alias('d')).collect()
+    [Row(d=datetime.date(2015, 4, 7))]
+    """
+    sc = SparkContext._active_spark_context
+    return Column(sc._jvm.functions.date_sub(_to_java_column(start), days))
+
+
+@since(1.5)
+def add_months(start, months):
+    """
+    Returns the date that is `months` months after `start`
+
+    >>> df = sqlContext.createDataFrame([('2015-04-08',)], ['d'])
+    >>> df.select(add_months(df.d, 1).alias('d')).collect()
+    [Row(d=datetime.date(2015, 5, 8))]
+    """
+    sc = SparkContext._active_spark_context
+    return Column(sc._jvm.functions.add_months(_to_java_column(start), months))
+
+
+@since(1.5)
+def months_between(date1, date2):
+    """
+    Returns the number of months between date1 and date2.
+
+    >>> df = sqlContext.createDataFrame([('1997-02-28 10:30:00', '1996-10-30')], ['t', 'd'])
+    >>> df.select(months_between(df.t, df.d).alias('months')).collect()
+    [Row(months=3.9495967...)]
+    """
+    sc = SparkContext._active_spark_context
+    return Column(sc._jvm.functions.months_between(_to_java_column(date1), _to_java_column(date2)))
 
 
 @since(1.5)

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index d663f12..6c7c481 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -205,9 +205,12 @@ object FunctionRegistry {
     expression[Upper]("upper"),
 
     // datetime functions
+    expression[AddMonths]("add_months"),
     expression[CurrentDate]("current_date"),
     expression[CurrentTimestamp]("current_timestamp"),
+    expression[DateAdd]("date_add"),
     expression[DateFormatClass]("date_format"),
+    expression[DateSub]("date_sub"),
     expression[DayOfMonth]("day"),
     expression[DayOfYear]("dayofyear"),
     expression[DayOfMonth]("dayofmonth"),
@@ -216,6 +219,7 @@ object FunctionRegistry {
     expression[LastDay]("last_day"),
     expression[Minute]("minute"),
     expression[Month]("month"),
+    expression[MonthsBetween]("months_between"),
     expression[NextDay]("next_day"),
     expression[Quarter]("quarter"),
     expression[Second]("second"),

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index ecc4898..603afc4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -47,6 +47,7 @@ object HiveTypeCoercion {
       Division ::
       PropagateTypes ::
       ImplicitTypeCasts ::
+      DateTimeOperations ::
       Nil
 
   // See https://cwiki.apache.org/confluence/display/Hive/LanguageManual+Types.
@@ -639,6 +640,27 @@ object HiveTypeCoercion {
   }
 
   /**
+   * Turns Add/Subtract of DateType/TimestampType/StringType and CalendarIntervalType
+   * to TimeAdd/TimeSub
+   */
+  object DateTimeOperations extends Rule[LogicalPlan] {
+
+    private val acceptedTypes = Seq(DateType, TimestampType, StringType)
+
+    def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
+      // Skip nodes who's children have not been resolved yet.
+      case e if !e.childrenResolved => e
+
+      case Add(l @ CalendarIntervalType(), r) if acceptedTypes.contains(r.dataType) =>
+        Cast(TimeAdd(r, l), r.dataType)
+      case Add(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) =>
+        Cast(TimeAdd(l, r), l.dataType)
+      case Subtract(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) =>
+        Cast(TimeSub(l, r), l.dataType)
+    }
+  }
+
+  /**
    * Casts types according to the expected input types for [[Expression]]s.
    */
   object ImplicitTypeCasts extends Rule[LogicalPlan] {

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala
index a5e6249..9795673 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String}
 
 import scala.util.Try
 
@@ -63,6 +63,53 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback {
   }
 }
 
+/**
+ * Adds a number of days to startdate.
+ */
+case class DateAdd(startDate: Expression, days: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def left: Expression = startDate
+  override def right: Expression = days
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType)
+
+  override def dataType: DataType = DateType
+
+  override def nullSafeEval(start: Any, d: Any): Any = {
+    start.asInstanceOf[Int] + d.asInstanceOf[Int]
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
+    nullSafeCodeGen(ctx, ev, (sd, d) => {
+      s"""${ev.primitive} = $sd + $d;"""
+    })
+  }
+}
+
+/**
+ * Subtracts a number of days to startdate.
+ */
+case class DateSub(startDate: Expression, days: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+  override def left: Expression = startDate
+  override def right: Expression = days
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType)
+
+  override def dataType: DataType = DateType
+
+  override def nullSafeEval(start: Any, d: Any): Any = {
+    start.asInstanceOf[Int] - d.asInstanceOf[Int]
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
+    nullSafeCodeGen(ctx, ev, (sd, d) => {
+      s"""${ev.primitive} = $sd - $d;"""
+    })
+  }
+}
+
 case class Hour(child: Expression) extends UnaryExpression with ImplicitCastInputTypes {
 
   override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType)
@@ -543,3 +590,109 @@ case class NextDay(startDate: Expression, dayOfWeek: Expression)
 
   override def prettyName: String = "next_day"
 }
+
+/**
+ * Adds an interval to timestamp.
+ */
+case class TimeAdd(start: Expression, interval: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def left: Expression = start
+  override def right: Expression = interval
+
+  override def toString: String = s"$left + $right"
+  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, CalendarIntervalType)
+
+  override def dataType: DataType = TimestampType
+
+  override def nullSafeEval(start: Any, interval: Any): Any = {
+    val itvl = interval.asInstanceOf[CalendarInterval]
+    DateTimeUtils.timestampAddInterval(
+      start.asInstanceOf[Long], itvl.months, itvl.microseconds)
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
+    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
+    defineCodeGen(ctx, ev, (sd, i) => {
+      s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds)"""
+    })
+  }
+}
+
+/**
+ * Subtracts an interval from timestamp.
+ */
+case class TimeSub(start: Expression, interval: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def left: Expression = start
+  override def right: Expression = interval
+
+  override def toString: String = s"$left - $right"
+  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, CalendarIntervalType)
+
+  override def dataType: DataType = TimestampType
+
+  override def nullSafeEval(start: Any, interval: Any): Any = {
+    val itvl = interval.asInstanceOf[CalendarInterval]
+    DateTimeUtils.timestampAddInterval(
+      start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds)
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
+    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
+    defineCodeGen(ctx, ev, (sd, i) => {
+      s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds)"""
+    })
+  }
+}
+
+/**
+ * Returns the date that is num_months after start_date.
+ */
+case class AddMonths(startDate: Expression, numMonths: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def left: Expression = startDate
+  override def right: Expression = numMonths
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(DateType, IntegerType)
+
+  override def dataType: DataType = DateType
+
+  override def nullSafeEval(start: Any, months: Any): Any = {
+    DateTimeUtils.dateAddMonths(start.asInstanceOf[Int], months.asInstanceOf[Int])
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
+    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
+    defineCodeGen(ctx, ev, (sd, m) => {
+      s"""$dtu.dateAddMonths($sd, $m)"""
+    })
+  }
+}
+
+/**
+ * Returns number of months between dates date1 and date2.
+ */
+case class MonthsBetween(date1: Expression, date2: Expression)
+  extends BinaryExpression with ImplicitCastInputTypes {
+
+  override def left: Expression = date1
+  override def right: Expression = date2
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, TimestampType)
+
+  override def dataType: DataType = DoubleType
+
+  override def nullSafeEval(t1: Any, t2: Any): Any = {
+    DateTimeUtils.monthsBetween(t1.asInstanceOf[Long], t2.asInstanceOf[Long])
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = {
+    val dtu = DateTimeUtils.getClass.getName.stripSuffix("$")
+    defineCodeGen(ctx, ev, (l, r) => {
+      s"""$dtu.monthsBetween($l, $r)"""
+    })
+  }
+}

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 93966a5..53abdf6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -45,6 +45,7 @@ object DateTimeUtils {
   final val to2001 = -11323
 
   // this is year -17999, calculation: 50 * daysIn400Year
+  final val YearZero = -17999
   final val toYearZero = to2001 + 7304850
 
   @transient lazy val defaultTimeZone = TimeZone.getDefault
@@ -575,6 +576,144 @@ object DateTimeUtils {
   }
 
   /**
+   * The number of days for each month (not leap year)
+   */
+  private val monthDays = Array(31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31)
+
+  /**
+   * Returns the date value for the first day of the given month.
+   * The month is expressed in months since year zero (17999 BC), starting from 0.
+   */
+  private def firstDayOfMonth(absoluteMonth: Int): Int = {
+    val absoluteYear = absoluteMonth / 12
+    var monthInYear = absoluteMonth - absoluteYear * 12
+    var date = getDateFromYear(absoluteYear)
+    if (monthInYear >= 2 && isLeapYear(absoluteYear + YearZero)) {
+      date += 1
+    }
+    while (monthInYear > 0) {
+      date += monthDays(monthInYear - 1)
+      monthInYear -= 1
+    }
+    date
+  }
+
+  /**
+   * Returns the date value for January 1 of the given year.
+   * The year is expressed in years since year zero (17999 BC), starting from 0.
+   */
+  private def getDateFromYear(absoluteYear: Int): Int = {
+    val absoluteDays = (absoluteYear * 365 + absoluteYear / 400 - absoluteYear / 100
+      + absoluteYear / 4)
+    absoluteDays - toYearZero
+  }
+
+  /**
+   * Add date and year-month interval.
+   * Returns a date value, expressed in days since 1.1.1970.
+   */
+  def dateAddMonths(days: Int, months: Int): Int = {
+    val absoluteMonth = (getYear(days) - YearZero) * 12 + getMonth(days) - 1 + months
+    val currentMonthInYear = absoluteMonth % 12
+    val currentYear = absoluteMonth / 12
+    val leapDay = if (currentMonthInYear == 1 && isLeapYear(currentYear + YearZero)) 1 else 0
+    val lastDayOfMonth = monthDays(currentMonthInYear) + leapDay
+
+    val dayOfMonth = getDayOfMonth(days)
+    val currentDayInMonth = if (getDayOfMonth(days + 1) == 1 || dayOfMonth >= lastDayOfMonth) {
+      // last day of the month
+      lastDayOfMonth
+    } else {
+      dayOfMonth
+    }
+    firstDayOfMonth(absoluteMonth) + currentDayInMonth - 1
+  }
+
+  /**
+   * Add timestamp and full interval.
+   * Returns a timestamp value, expressed in microseconds since 1.1.1970 00:00:00.
+   */
+  def timestampAddInterval(start: Long, months: Int, microseconds: Long): Long = {
+    val days = millisToDays(start / 1000L)
+    val newDays = dateAddMonths(days, months)
+    daysToMillis(newDays) * 1000L + start - daysToMillis(days) * 1000L + microseconds
+  }
+
+  /**
+   * Returns the last dayInMonth in the month it belongs to. The date is expressed
+   * in days since 1.1.1970. the return value starts from 1.
+   */
+  private def getLastDayInMonthOfMonth(date: Int): Int = {
+    var (year, dayInYear) = getYearAndDayInYear(date)
+    if (isLeapYear(year)) {
+      if (dayInYear > 31 && dayInYear <= 60) {
+        return 29
+      } else if (dayInYear > 60) {
+        dayInYear = dayInYear - 1
+      }
+    }
+    if (dayInYear <= 31) {
+      31
+    } else if (dayInYear <= 59) {
+      28
+    } else if (dayInYear <= 90) {
+      31
+    } else if (dayInYear <= 120) {
+      30
+    } else if (dayInYear <= 151) {
+      31
+    } else if (dayInYear <= 181) {
+      30
+    } else if (dayInYear <= 212) {
+      31
+    } else if (dayInYear <= 243) {
+      31
+    } else if (dayInYear <= 273) {
+      30
+    } else if (dayInYear <= 304) {
+      31
+    } else if (dayInYear <= 334) {
+      30
+    } else {
+      31
+    }
+  }
+
+  /**
+   * Returns number of months between time1 and time2. time1 and time2 are expressed in
+   * microseconds since 1.1.1970.
+   *
+   * If time1 and time2 having the same day of month, or both are the last day of month,
+   * it returns an integer (time under a day will be ignored).
+   *
+   * Otherwise, the difference is calculated based on 31 days per month, and rounding to
+   * 8 digits.
+   */
+  def monthsBetween(time1: Long, time2: Long): Double = {
+    val millis1 = time1 / 1000L
+    val millis2 = time2 / 1000L
+    val date1 = millisToDays(millis1)
+    val date2 = millisToDays(millis2)
+    // TODO(davies): get year, month, dayOfMonth from single function
+    val dayInMonth1 = getDayOfMonth(date1)
+    val dayInMonth2 = getDayOfMonth(date2)
+    val months1 = getYear(date1) * 12 + getMonth(date1)
+    val months2 = getYear(date2) * 12 + getMonth(date2)
+
+    if (dayInMonth1 == dayInMonth2 || (dayInMonth1 == getLastDayInMonthOfMonth(date1)
+      && dayInMonth2 == getLastDayInMonthOfMonth(date2))) {
+      return (months1 - months2).toDouble
+    }
+    // milliseconds is enough for 8 digits precision on the right side
+    val timeInDay1 = millis1 - daysToMillis(date1)
+    val timeInDay2 = millis2 - daysToMillis(date2)
+    val timesBetween = (timeInDay1 - timeInDay2).toDouble / MILLIS_PER_DAY
+    val diff = (months1 - months2).toDouble + (dayInMonth1 - dayInMonth2 + timesBetween) / 31.0
+    // rounding to 8 digits
+    math.round(diff * 1e8) / 1e8
+  }
+
+  /*
    * Returns day of week from String. Starting from Thursday, marked as 0.
    * (Because 1970-01-01 is Thursday).
    */

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
index 1d9ee5d..7060877 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
@@ -17,12 +17,15 @@
 
 package org.apache.spark.sql.catalyst.analysis
 
+import java.sql.Timestamp
+
 import org.apache.spark.sql.catalyst.plans.PlanTest
 
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.rules.Rule
 import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.CalendarInterval
 
 class HiveTypeCoercionSuite extends PlanTest {
 
@@ -400,6 +403,33 @@ class HiveTypeCoercionSuite extends PlanTest {
     }
   }
 
+  test("rule for date/timestamp operations") {
+    val dateTimeOperations = HiveTypeCoercion.DateTimeOperations
+    val date = Literal(new java.sql.Date(0L))
+    val timestamp = Literal(new Timestamp(0L))
+    val interval = Literal(new CalendarInterval(0, 0))
+    val str = Literal("2015-01-01")
+
+    ruleTest(dateTimeOperations, Add(date, interval), Cast(TimeAdd(date, interval), DateType))
+    ruleTest(dateTimeOperations, Add(interval, date), Cast(TimeAdd(date, interval), DateType))
+    ruleTest(dateTimeOperations, Add(timestamp, interval),
+      Cast(TimeAdd(timestamp, interval), TimestampType))
+    ruleTest(dateTimeOperations, Add(interval, timestamp),
+      Cast(TimeAdd(timestamp, interval), TimestampType))
+    ruleTest(dateTimeOperations, Add(str, interval), Cast(TimeAdd(str, interval), StringType))
+    ruleTest(dateTimeOperations, Add(interval, str), Cast(TimeAdd(str, interval), StringType))
+
+    ruleTest(dateTimeOperations, Subtract(date, interval), Cast(TimeSub(date, interval), DateType))
+    ruleTest(dateTimeOperations, Subtract(timestamp, interval),
+      Cast(TimeSub(timestamp, interval), TimestampType))
+    ruleTest(dateTimeOperations, Subtract(str, interval), Cast(TimeSub(str, interval), StringType))
+
+    // interval operations should not be effected
+    ruleTest(dateTimeOperations, Add(interval, interval), Add(interval, interval))
+    ruleTest(dateTimeOperations, Subtract(interval, interval), Subtract(interval, interval))
+  }
+
+
   /**
    * There are rules that need to not fire before child expressions get resolved.
    * We use this test to make sure those rules do not fire early.

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
index e1387f9..fd1d6c1 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DateExpressionsSuite.scala
@@ -22,8 +22,8 @@ import java.text.SimpleDateFormat
 import java.util.Calendar
 
 import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.unsafe.types.CalendarInterval
 import org.apache.spark.sql.types._
 
 class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
@@ -48,56 +48,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
 
   test("DayOfYear") {
     val sdfDay = new SimpleDateFormat("D")
-    (2002 to 2004).foreach { y =>
-      (0 to 11).foreach { m =>
-        (0 to 5).foreach { i =>
-          val c = Calendar.getInstance()
-          c.set(y, m, 28, 0, 0, 0)
-          c.add(Calendar.DATE, i)
-          checkEvaluation(DayOfYear(Literal(new Date(c.getTimeInMillis))),
-            sdfDay.format(c.getTime).toInt)
-        }
-      }
-    }
-
     (1998 to 2002).foreach { y =>
-      (0 to 11).foreach { m =>
-        (0 to 5).foreach { i =>
-          val c = Calendar.getInstance()
-          c.set(y, m, 28, 0, 0, 0)
-          c.add(Calendar.DATE, i)
-          checkEvaluation(DayOfYear(Literal(new Date(c.getTimeInMillis))),
-            sdfDay.format(c.getTime).toInt)
-        }
-      }
-    }
-
-    (1969 to 1970).foreach { y =>
-      (0 to 11).foreach { m =>
-        (0 to 5).foreach { i =>
-          val c = Calendar.getInstance()
-          c.set(y, m, 28, 0, 0, 0)
-          c.add(Calendar.DATE, i)
-          checkEvaluation(DayOfYear(Literal(new Date(c.getTimeInMillis))),
-            sdfDay.format(c.getTime).toInt)
-        }
-      }
-    }
-
-    (2402 to 2404).foreach { y =>
-      (0 to 11).foreach { m =>
-        (0 to 5).foreach { i =>
-          val c = Calendar.getInstance()
-          c.set(y, m, 28, 0, 0, 0)
-          c.add(Calendar.DATE, i)
-          checkEvaluation(DayOfYear(Literal(new Date(c.getTimeInMillis))),
-            sdfDay.format(c.getTime).toInt)
-        }
-      }
-    }
-
-    (2398 to 2402).foreach { y =>
-      (0 to 11).foreach { m =>
+      (0 to 3).foreach { m =>
         (0 to 5).foreach { i =>
           val c = Calendar.getInstance()
           c.set(y, m, 28, 0, 0, 0)
@@ -117,7 +69,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(Year(Cast(Literal(ts), DateType)), 2013)
 
     val c = Calendar.getInstance()
-    (2000 to 2010).foreach { y =>
+    (2000 to 2002).foreach { y =>
       (0 to 11 by 11).foreach { m =>
         c.set(y, m, 28)
         (0 to 5 * 24).foreach { i =>
@@ -155,20 +107,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     checkEvaluation(Month(Cast(Literal(ts), DateType)), 11)
 
     (2003 to 2004).foreach { y =>
-      (0 to 11).foreach { m =>
-        (0 to 5 * 24).foreach { i =>
-          val c = Calendar.getInstance()
-          c.set(y, m, 28, 0, 0, 0)
-          c.add(Calendar.HOUR_OF_DAY, i)
-          checkEvaluation(Month(Literal(new Date(c.getTimeInMillis))),
-            c.get(Calendar.MONTH) + 1)
-        }
-      }
-    }
-
-    (1999 to 2000).foreach { y =>
-      (0 to 11).foreach { m =>
-        (0 to 5 * 24).foreach { i =>
+      (0 to 3).foreach { m =>
+        (0 to 2 * 24).foreach { i =>
           val c = Calendar.getInstance()
           c.set(y, m, 28, 0, 0, 0)
           c.add(Calendar.HOUR_OF_DAY, i)
@@ -262,6 +202,112 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper {
     }
   }
 
+  test("date_add") {
+    checkEvaluation(
+      DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(1)),
+      DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29")))
+    checkEvaluation(
+      DateAdd(Literal(Date.valueOf("2016-02-28")), Literal(-365)),
+      DateTimeUtils.fromJavaDate(Date.valueOf("2015-02-28")))
+    checkEvaluation(DateAdd(Literal.create(null, DateType), Literal(1)), null)
+    checkEvaluation(DateAdd(Literal(Date.valueOf("2016-02-28")), Literal.create(null, IntegerType)),
+      null)
+    checkEvaluation(DateAdd(Literal.create(null, DateType), Literal.create(null, IntegerType)),
+      null)
+  }
+
+  test("date_sub") {
+    checkEvaluation(
+      DateSub(Literal(Date.valueOf("2015-01-01")), Literal(1)),
+      DateTimeUtils.fromJavaDate(Date.valueOf("2014-12-31")))
+    checkEvaluation(
+      DateSub(Literal(Date.valueOf("2015-01-01")), Literal(-1)),
+      DateTimeUtils.fromJavaDate(Date.valueOf("2015-01-02")))
+    checkEvaluation(DateSub(Literal.create(null, DateType), Literal(1)), null)
+    checkEvaluation(DateSub(Literal(Date.valueOf("2016-02-28")), Literal.create(null, IntegerType)),
+      null)
+    checkEvaluation(DateSub(Literal.create(null, DateType), Literal.create(null, IntegerType)),
+      null)
+  }
+
+  test("time_add") {
+    checkEvaluation(
+      TimeAdd(Literal(Timestamp.valueOf("2016-01-29 10:00:00")),
+        Literal(new CalendarInterval(1, 123000L))),
+      DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2016-02-29 10:00:00.123")))
+
+    checkEvaluation(
+      TimeAdd(Literal.create(null, TimestampType), Literal(new CalendarInterval(1, 123000L))),
+      null)
+    checkEvaluation(
+      TimeAdd(Literal(Timestamp.valueOf("2016-01-29 10:00:00")),
+        Literal.create(null, CalendarIntervalType)),
+      null)
+    checkEvaluation(
+      TimeAdd(Literal.create(null, TimestampType), Literal.create(null, CalendarIntervalType)),
+      null)
+  }
+
+  test("time_sub") {
+    checkEvaluation(
+      TimeSub(Literal(Timestamp.valueOf("2016-03-31 10:00:00")),
+        Literal(new CalendarInterval(1, 0))),
+      DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2016-02-29 10:00:00")))
+    checkEvaluation(
+      TimeSub(
+        Literal(Timestamp.valueOf("2016-03-30 00:00:01")),
+        Literal(new CalendarInterval(1, 2000000.toLong))),
+      DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2016-02-28 23:59:59")))
+
+    checkEvaluation(
+      TimeSub(Literal.create(null, TimestampType), Literal(new CalendarInterval(1, 123000L))),
+      null)
+    checkEvaluation(
+      TimeSub(Literal(Timestamp.valueOf("2016-01-29 10:00:00")),
+        Literal.create(null, CalendarIntervalType)),
+      null)
+    checkEvaluation(
+      TimeSub(Literal.create(null, TimestampType), Literal.create(null, CalendarIntervalType)),
+      null)
+  }
+
+  test("add_months") {
+    checkEvaluation(AddMonths(Literal(Date.valueOf("2015-01-30")), Literal(1)),
+      DateTimeUtils.fromJavaDate(Date.valueOf("2015-02-28")))
+    checkEvaluation(AddMonths(Literal(Date.valueOf("2016-03-30")), Literal(-1)),
+      DateTimeUtils.fromJavaDate(Date.valueOf("2016-02-29")))
+    checkEvaluation(
+      AddMonths(Literal(Date.valueOf("2015-01-30")), Literal.create(null, IntegerType)),
+      null)
+    checkEvaluation(AddMonths(Literal.create(null, DateType), Literal(1)), null)
+    checkEvaluation(AddMonths(Literal.create(null, DateType), Literal.create(null, IntegerType)),
+      null)
+  }
+
+  test("months_between") {
+    checkEvaluation(
+      MonthsBetween(Literal(Timestamp.valueOf("1997-02-28 10:30:00")),
+        Literal(Timestamp.valueOf("1996-10-30 00:00:00"))),
+      3.94959677)
+    checkEvaluation(
+      MonthsBetween(Literal(Timestamp.valueOf("2015-01-30 11:52:00")),
+        Literal(Timestamp.valueOf("2015-01-30 11:50:00"))),
+      0.0)
+    checkEvaluation(
+      MonthsBetween(Literal(Timestamp.valueOf("2015-01-31 00:00:00")),
+        Literal(Timestamp.valueOf("2015-03-31 22:00:00"))),
+      -2.0)
+    checkEvaluation(
+      MonthsBetween(Literal(Timestamp.valueOf("2015-03-31 22:00:00")),
+        Literal(Timestamp.valueOf("2015-02-28 00:00:00"))),
+      1.0)
+    val t = Literal(Timestamp.valueOf("2015-03-31 22:00:00"))
+    val tnull = Literal.create(null, TimestampType)
+    checkEvaluation(MonthsBetween(t, tnull), null)
+    checkEvaluation(MonthsBetween(tnull, t), null)
+    checkEvaluation(MonthsBetween(tnull, tnull), null)
+  }
+
   test("last_day") {
     checkEvaluation(LastDay(Literal(Date.valueOf("2015-02-28"))), Date.valueOf("2015-02-28"))
     checkEvaluation(LastDay(Literal(Date.valueOf("2015-03-27"))), Date.valueOf("2015-03-31"))

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
index fab9eb9..60d2bcf 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala
@@ -19,47 +19,48 @@ package org.apache.spark.sql.catalyst.util
 
 import java.sql.{Date, Timestamp}
 import java.text.SimpleDateFormat
-import java.util.{TimeZone, Calendar}
+import java.util.{Calendar, TimeZone}
 
 import org.apache.spark.SparkFunSuite
 import org.apache.spark.unsafe.types.UTF8String
+import org.apache.spark.sql.catalyst.util.DateTimeUtils._
 
 class DateTimeUtilsSuite extends SparkFunSuite {
 
   private[this] def getInUTCDays(timestamp: Long): Int = {
     val tz = TimeZone.getDefault
-    ((timestamp + tz.getOffset(timestamp)) / DateTimeUtils.MILLIS_PER_DAY).toInt
+    ((timestamp + tz.getOffset(timestamp)) / MILLIS_PER_DAY).toInt
   }
 
   test("timestamp and us") {
     val now = new Timestamp(System.currentTimeMillis())
     now.setNanos(1000)
-    val ns = DateTimeUtils.fromJavaTimestamp(now)
+    val ns = fromJavaTimestamp(now)
     assert(ns % 1000000L === 1)
-    assert(DateTimeUtils.toJavaTimestamp(ns) === now)
+    assert(toJavaTimestamp(ns) === now)
 
     List(-111111111111L, -1L, 0, 1L, 111111111111L).foreach { t =>
-      val ts = DateTimeUtils.toJavaTimestamp(t)
-      assert(DateTimeUtils.fromJavaTimestamp(ts) === t)
-      assert(DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromJavaTimestamp(ts)) === ts)
+      val ts = toJavaTimestamp(t)
+      assert(fromJavaTimestamp(ts) === t)
+      assert(toJavaTimestamp(fromJavaTimestamp(ts)) === ts)
     }
   }
 
   test("us and julian day") {
-    val (d, ns) = DateTimeUtils.toJulianDay(0)
-    assert(d === DateTimeUtils.JULIAN_DAY_OF_EPOCH)
-    assert(ns === DateTimeUtils.SECONDS_PER_DAY / 2 * DateTimeUtils.NANOS_PER_SECOND)
-    assert(DateTimeUtils.fromJulianDay(d, ns) == 0L)
+    val (d, ns) = toJulianDay(0)
+    assert(d === JULIAN_DAY_OF_EPOCH)
+    assert(ns === SECONDS_PER_DAY / 2 * NANOS_PER_SECOND)
+    assert(fromJulianDay(d, ns) == 0L)
 
     val t = new Timestamp(61394778610000L) // (2015, 6, 11, 10, 10, 10, 100)
-    val (d1, ns1) = DateTimeUtils.toJulianDay(DateTimeUtils.fromJavaTimestamp(t))
-    val t2 = DateTimeUtils.toJavaTimestamp(DateTimeUtils.fromJulianDay(d1, ns1))
+    val (d1, ns1) = toJulianDay(fromJavaTimestamp(t))
+    val t2 = toJavaTimestamp(fromJulianDay(d1, ns1))
     assert(t.equals(t2))
   }
 
   test("SPARK-6785: java date conversion before and after epoch") {
     def checkFromToJavaDate(d1: Date): Unit = {
-      val d2 = DateTimeUtils.toJavaDate(DateTimeUtils.fromJavaDate(d1))
+      val d2 = toJavaDate(fromJavaDate(d1))
       assert(d2.toString === d1.toString)
     }
 
@@ -95,157 +96,156 @@ class DateTimeUtilsSuite extends SparkFunSuite {
   }
 
   test("string to date") {
-    import DateTimeUtils.millisToDays
 
     var c = Calendar.getInstance()
     c.set(2015, 0, 28, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-01-28")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-01-28")).get ===
       millisToDays(c.getTimeInMillis))
     c.set(2015, 0, 1, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015")).get ===
+    assert(stringToDate(UTF8String.fromString("2015")).get ===
       millisToDays(c.getTimeInMillis))
     c = Calendar.getInstance()
     c.set(2015, 2, 1, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-03")).get ===
       millisToDays(c.getTimeInMillis))
     c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03-18")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-03-18")).get ===
       millisToDays(c.getTimeInMillis))
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03-18 ")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-03-18 ")).get ===
       millisToDays(c.getTimeInMillis))
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03-18 123142")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-03-18 123142")).get ===
       millisToDays(c.getTimeInMillis))
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03-18T123123")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-03-18T123123")).get ===
       millisToDays(c.getTimeInMillis))
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03-18T")).get ===
+    assert(stringToDate(UTF8String.fromString("2015-03-18T")).get ===
       millisToDays(c.getTimeInMillis))
 
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-03-18X")).isEmpty)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015/03/18")).isEmpty)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015.03.18")).isEmpty)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("20150318")).isEmpty)
-    assert(DateTimeUtils.stringToDate(UTF8String.fromString("2015-031-8")).isEmpty)
+    assert(stringToDate(UTF8String.fromString("2015-03-18X")).isEmpty)
+    assert(stringToDate(UTF8String.fromString("2015/03/18")).isEmpty)
+    assert(stringToDate(UTF8String.fromString("2015.03.18")).isEmpty)
+    assert(stringToDate(UTF8String.fromString("20150318")).isEmpty)
+    assert(stringToDate(UTF8String.fromString("2015-031-8")).isEmpty)
   }
 
   test("string to timestamp") {
     var c = Calendar.getInstance()
     c.set(1969, 11, 31, 16, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("1969-12-31 16:00:00")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("1969-12-31 16:00:00")).get ===
       c.getTimeInMillis * 1000)
     c.set(2015, 0, 1, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015")).get ===
       c.getTimeInMillis * 1000)
     c = Calendar.getInstance()
     c.set(2015, 2, 1, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03")).get ===
       c.getTimeInMillis * 1000)
     c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18")).get ===
       c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18 ")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18 ")).get ===
       c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18T")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18T")).get ===
       c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance()
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18 12:03:17")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18 12:03:17")).get ===
       c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17")).get ===
       c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT-13:53"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17-13:53")).get === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17Z")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17Z")).get ===
       c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18 12:03:17Z")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18 12:03:17Z")).get ===
       c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17-1:0")).get ===
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18T12:03:17-1:0")).get ===
       c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17-01:00")).get === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17+07:30")).get === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17+07:03")).get === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance()
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18 12:03:17.123")).get === c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.123")).get === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("UTC"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 456)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.456Z")).get  === c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18 12:03:17.456Z")).get  === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.123-1:0")).get  === c.getTimeInMillis * 1000)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.123-01:00")).get ===  c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.123+07:30")).get ===  c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.123+07:30")).get === c.getTimeInMillis * 1000)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.123121+7:30")).get ===
         c.getTimeInMillis * 1000 + 121)
 
     c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30"))
     c.set(2015, 2, 18, 12, 3, 17)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03:17.12312+7:30")).get ===
         c.getTimeInMillis * 1000 + 120)
 
@@ -254,7 +254,7 @@ class DateTimeUtilsSuite extends SparkFunSuite {
     c.set(Calendar.MINUTE, 12)
     c.set(Calendar.SECOND, 15)
     c.set(Calendar.MILLISECOND, 0)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("18:12:15")).get ===
       c.getTimeInMillis * 1000)
 
@@ -263,7 +263,7 @@ class DateTimeUtilsSuite extends SparkFunSuite {
     c.set(Calendar.MINUTE, 12)
     c.set(Calendar.SECOND, 15)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("T18:12:15.12312+7:30")).get ===
       c.getTimeInMillis * 1000 + 120)
 
@@ -272,93 +272,130 @@ class DateTimeUtilsSuite extends SparkFunSuite {
     c.set(Calendar.MINUTE, 12)
     c.set(Calendar.SECOND, 15)
     c.set(Calendar.MILLISECOND, 123)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("18:12:15.12312+7:30")).get ===
       c.getTimeInMillis * 1000 + 120)
 
     c = Calendar.getInstance()
     c.set(2011, 4, 6, 7, 8, 9)
     c.set(Calendar.MILLISECOND, 100)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2011-05-06 07:08:09.1000")).get === c.getTimeInMillis * 1000)
 
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("238")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18 123142")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18T123123")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-18X")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015/03/18")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015.03.18")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("20150318")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-031-8")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(UTF8String.fromString("238")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18 123142")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18T123123")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("2015-03-18X")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("2015/03/18")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("2015.03.18")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("20150318")).isEmpty)
+    assert(stringToTimestamp(UTF8String.fromString("2015-031-8")).isEmpty)
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03.17-20:0")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03.17-0:70")).isEmpty)
-    assert(DateTimeUtils.stringToTimestamp(
+    assert(stringToTimestamp(
       UTF8String.fromString("2015-03-18T12:03.17-1:0:0")).isEmpty)
   }
 
   test("hours") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 13, 2, 11)
-    assert(DateTimeUtils.getHours(c.getTimeInMillis * 1000) === 13)
+    assert(getHours(c.getTimeInMillis * 1000) === 13)
     c.set(2015, 12, 8, 2, 7, 9)
-    assert(DateTimeUtils.getHours(c.getTimeInMillis * 1000) === 2)
+    assert(getHours(c.getTimeInMillis * 1000) === 2)
   }
 
   test("minutes") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 13, 2, 11)
-    assert(DateTimeUtils.getMinutes(c.getTimeInMillis * 1000) === 2)
+    assert(getMinutes(c.getTimeInMillis * 1000) === 2)
     c.set(2015, 2, 8, 2, 7, 9)
-    assert(DateTimeUtils.getMinutes(c.getTimeInMillis * 1000) === 7)
+    assert(getMinutes(c.getTimeInMillis * 1000) === 7)
   }
 
   test("seconds") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 13, 2, 11)
-    assert(DateTimeUtils.getSeconds(c.getTimeInMillis * 1000) === 11)
+    assert(getSeconds(c.getTimeInMillis * 1000) === 11)
     c.set(2015, 2, 8, 2, 7, 9)
-    assert(DateTimeUtils.getSeconds(c.getTimeInMillis * 1000) === 9)
+    assert(getSeconds(c.getTimeInMillis * 1000) === 9)
   }
 
   test("get day in year") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getDayInYear(getInUTCDays(c.getTimeInMillis)) === 77)
+    assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 77)
     c.set(2012, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getDayInYear(getInUTCDays(c.getTimeInMillis)) === 78)
+    assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 78)
   }
 
   test("get year") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getYear(getInUTCDays(c.getTimeInMillis)) === 2015)
+    assert(getYear(getInUTCDays(c.getTimeInMillis)) === 2015)
     c.set(2012, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getYear(getInUTCDays(c.getTimeInMillis)) === 2012)
+    assert(getYear(getInUTCDays(c.getTimeInMillis)) === 2012)
   }
 
   test("get quarter") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getQuarter(getInUTCDays(c.getTimeInMillis)) === 1)
+    assert(getQuarter(getInUTCDays(c.getTimeInMillis)) === 1)
     c.set(2012, 11, 18, 0, 0, 0)
-    assert(DateTimeUtils.getQuarter(getInUTCDays(c.getTimeInMillis)) === 4)
+    assert(getQuarter(getInUTCDays(c.getTimeInMillis)) === 4)
   }
 
   test("get month") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getMonth(getInUTCDays(c.getTimeInMillis)) === 3)
+    assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 3)
     c.set(2012, 11, 18, 0, 0, 0)
-    assert(DateTimeUtils.getMonth(getInUTCDays(c.getTimeInMillis)) === 12)
+    assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 12)
   }
 
   test("get day of month") {
     val c = Calendar.getInstance()
     c.set(2015, 2, 18, 0, 0, 0)
-    assert(DateTimeUtils.getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 18)
+    assert(getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 18)
     c.set(2012, 11, 24, 0, 0, 0)
-    assert(DateTimeUtils.getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 24)
+    assert(getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 24)
+  }
+
+  test("date add months") {
+    val c1 = Calendar.getInstance()
+    c1.set(1997, 1, 28, 10, 30, 0)
+    val days1 = millisToDays(c1.getTimeInMillis)
+    val c2 = Calendar.getInstance()
+    c2.set(2000, 1, 29)
+    assert(dateAddMonths(days1, 36) === millisToDays(c2.getTimeInMillis))
+    c2.set(1996, 0, 31)
+    assert(dateAddMonths(days1, -13) === millisToDays(c2.getTimeInMillis))
+  }
+
+  test("timestamp add months") {
+    val c1 = Calendar.getInstance()
+    c1.set(1997, 1, 28, 10, 30, 0)
+    c1.set(Calendar.MILLISECOND, 0)
+    val ts1 = c1.getTimeInMillis * 1000L
+    val c2 = Calendar.getInstance()
+    c2.set(2000, 1, 29, 10, 30, 0)
+    c2.set(Calendar.MILLISECOND, 123)
+    val ts2 = c2.getTimeInMillis * 1000L
+    assert(timestampAddInterval(ts1, 36, 123000) === ts2)
+  }
+
+  test("monthsBetween") {
+    val c1 = Calendar.getInstance()
+    c1.set(1997, 1, 28, 10, 30, 0)
+    val c2 = Calendar.getInstance()
+    c2.set(1996, 9, 30, 0, 0, 0)
+    assert(monthsBetween(c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L) === 3.94959677)
+    c2.set(2000, 1, 28, 0, 0, 0)
+    assert(monthsBetween(c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L) === -36)
+    c2.set(2000, 1, 29, 0, 0, 0)
+    assert(monthsBetween(c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L) === -36)
+    c2.set(1996, 2, 31, 0, 0, 0)
+    assert(monthsBetween(c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L) === 11)
   }
 }

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/core/src/main/scala/org/apache/spark/sql/functions.scala
----------------------------------------------------------------------
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 3f440e0..168894d 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
@@ -1928,6 +1928,14 @@ object functions {
   //////////////////////////////////////////////////////////////////////////////////////////////
 
   /**
+   * Returns the date that is numMonths after startDate.
+   * @group datetime_funcs
+   * @since 1.5.0
+   */
+  def add_months(startDate: Column, numMonths: Int): Column =
+    AddMonths(startDate.expr, Literal(numMonths))
+
+  /**
    * Converts a date/timestamp/string to a value of string in the format specified by the date
    * format given by the second argument.
    *
@@ -1960,6 +1968,20 @@ object functions {
     date_format(Column(dateColumnName), format)
 
   /**
+   * Returns the date that is `days` days after `start`
+   * @group datetime_funcs
+   * @since 1.5.0
+   */
+  def date_add(start: Column, days: Int): Column = DateAdd(start.expr, Literal(days))
+
+  /**
+   * Returns the date that is `days` days before `start`
+   * @group datetime_funcs
+   * @since 1.5.0
+   */
+  def date_sub(start: Column, days: Int): Column = DateSub(start.expr, Literal(days))
+
+  /**
    * Extracts the year as an integer from a given date/timestamp/string.
    * @group datetime_funcs
    * @since 1.5.0
@@ -2067,6 +2089,13 @@ object functions {
    */
   def minute(columnName: String): Column = minute(Column(columnName))
 
+  /*
+   * Returns number of months between dates `date1` and `date2`.
+   * @group datetime_funcs
+   * @since 1.5.0
+   */
+  def months_between(date1: Column, date2: Column): Column = MonthsBetween(date1.expr, date2.expr)
+
   /**
    * Given a date column, returns the first date which is later than the value of the date column
    * that is on the specified day of the week.

http://git-wip-us.apache.org/repos/asf/spark/blob/1abf7dc1/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
index df4cb57..b7267c4 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala
@@ -22,6 +22,7 @@ import java.text.SimpleDateFormat
 
 import org.apache.spark.sql.catalyst.util.DateTimeUtils
 import org.apache.spark.sql.functions._
+import org.apache.spark.unsafe.types.CalendarInterval
 
 class DateFunctionsSuite extends QueryTest {
   private lazy val ctx = org.apache.spark.sql.test.TestSQLContext
@@ -206,6 +207,122 @@ class DateFunctionsSuite extends QueryTest {
       Row(15, 15, 15))
   }
 
+  test("function date_add") {
+    val st1 = "2015-06-01 12:34:56"
+    val st2 = "2015-06-02 12:34:56"
+    val t1 = Timestamp.valueOf(st1)
+    val t2 = Timestamp.valueOf(st2)
+    val s1 = "2015-06-01"
+    val s2 = "2015-06-02"
+    val d1 = Date.valueOf(s1)
+    val d2 = Date.valueOf(s2)
+    val df = Seq((t1, d1, s1, st1), (t2, d2, s2, st2)).toDF("t", "d", "s", "ss")
+    checkAnswer(
+      df.select(date_add(col("d"), 1)),
+      Seq(Row(Date.valueOf("2015-06-02")), Row(Date.valueOf("2015-06-03"))))
+    checkAnswer(
+      df.select(date_add(col("t"), 3)),
+      Seq(Row(Date.valueOf("2015-06-04")), Row(Date.valueOf("2015-06-05"))))
+    checkAnswer(
+      df.select(date_add(col("s"), 5)),
+      Seq(Row(Date.valueOf("2015-06-06")), Row(Date.valueOf("2015-06-07"))))
+    checkAnswer(
+      df.select(date_add(col("ss"), 7)),
+      Seq(Row(Date.valueOf("2015-06-08")), Row(Date.valueOf("2015-06-09"))))
+
+    checkAnswer(df.selectExpr("DATE_ADD(null, 1)"), Seq(Row(null), Row(null)))
+    checkAnswer(
+      df.selectExpr("""DATE_ADD(d, 1)"""),
+      Seq(Row(Date.valueOf("2015-06-02")), Row(Date.valueOf("2015-06-03"))))
+  }
+
+  test("function date_sub") {
+    val st1 = "2015-06-01 12:34:56"
+    val st2 = "2015-06-02 12:34:56"
+    val t1 = Timestamp.valueOf(st1)
+    val t2 = Timestamp.valueOf(st2)
+    val s1 = "2015-06-01"
+    val s2 = "2015-06-02"
+    val d1 = Date.valueOf(s1)
+    val d2 = Date.valueOf(s2)
+    val df = Seq((t1, d1, s1, st1), (t2, d2, s2, st2)).toDF("t", "d", "s", "ss")
+    checkAnswer(
+      df.select(date_sub(col("d"), 1)),
+      Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01"))))
+    checkAnswer(
+      df.select(date_sub(col("t"), 1)),
+      Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01"))))
+    checkAnswer(
+      df.select(date_sub(col("s"), 1)),
+      Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01"))))
+    checkAnswer(
+      df.select(date_sub(col("ss"), 1)),
+      Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01"))))
+    checkAnswer(
+      df.select(date_sub(lit(null), 1)).limit(1), Row(null))
+
+    checkAnswer(df.selectExpr("""DATE_SUB(d, null)"""), Seq(Row(null), Row(null)))
+    checkAnswer(
+      df.selectExpr("""DATE_SUB(d, 1)"""),
+      Seq(Row(Date.valueOf("2015-05-31")), Row(Date.valueOf("2015-06-01"))))
+  }
+
+  test("time_add") {
+    val t1 = Timestamp.valueOf("2015-07-31 23:59:59")
+    val t2 = Timestamp.valueOf("2015-12-31 00:00:00")
+    val d1 = Date.valueOf("2015-07-31")
+    val d2 = Date.valueOf("2015-12-31")
+    val i = new CalendarInterval(2, 2000000L)
+    val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d")
+    checkAnswer(
+      df.selectExpr(s"d + $i"),
+      Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2016-02-29"))))
+    checkAnswer(
+      df.selectExpr(s"t + $i"),
+      Seq(Row(Timestamp.valueOf("2015-10-01 00:00:01")),
+        Row(Timestamp.valueOf("2016-02-29 00:00:02"))))
+  }
+
+  test("time_sub") {
+    val t1 = Timestamp.valueOf("2015-10-01 00:00:01")
+    val t2 = Timestamp.valueOf("2016-02-29 00:00:02")
+    val d1 = Date.valueOf("2015-09-30")
+    val d2 = Date.valueOf("2016-02-29")
+    val i = new CalendarInterval(2, 2000000L)
+    val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d")
+    checkAnswer(
+      df.selectExpr(s"d - $i"),
+      Seq(Row(Date.valueOf("2015-07-30")), Row(Date.valueOf("2015-12-30"))))
+    checkAnswer(
+      df.selectExpr(s"t - $i"),
+      Seq(Row(Timestamp.valueOf("2015-07-31 23:59:59")),
+        Row(Timestamp.valueOf("2015-12-31 00:00:00"))))
+  }
+
+  test("function add_months") {
+    val d1 = Date.valueOf("2015-08-31")
+    val d2 = Date.valueOf("2015-02-28")
+    val df = Seq((1, d1), (2, d2)).toDF("n", "d")
+    checkAnswer(
+      df.select(add_months(col("d"), 1)),
+      Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-31"))))
+    checkAnswer(
+      df.selectExpr("add_months(d, -1)"),
+      Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-31"))))
+  }
+
+  test("function months_between") {
+    val d1 = Date.valueOf("2015-07-31")
+    val d2 = Date.valueOf("2015-02-16")
+    val t1 = Timestamp.valueOf("2014-09-30 23:30:00")
+    val t2 = Timestamp.valueOf("2015-09-16 12:00:00")
+    val s1 = "2014-09-15 11:30:00"
+    val s2 = "2015-10-01 00:00:00"
+    val df = Seq((t1, d1, s1), (t2, d2, s2)).toDF("t", "d", "s")
+    checkAnswer(df.select(months_between(col("t"), col("d"))), Seq(Row(-10.0), Row(7.0)))
+    checkAnswer(df.selectExpr("months_between(t, s)"), Seq(Row(0.5), Row(-0.5)))
+  }
+
   test("function last_day") {
     val df1 = Seq((1, "2015-07-23"), (2, "2015-07-24")).toDF("i", "d")
     val df2 = Seq((1, "2015-07-23 00:11:22"), (2, "2015-07-24 11:22:33")).toDF("i", "t")


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