You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@impala.apache.org by ta...@apache.org on 2019/06/26 06:12:33 UTC

[impala] 02/04: IMPALA-7369: part 2: Add INTERVAL expr support and built-in functions for DATE

This is an automated email from the ASF dual-hosted git repository.

tarmstrong pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/impala.git

commit f40935a30e04576dafef03701e75cdffd4837a41
Author: Attila Jeges <at...@cloudera.com>
AuthorDate: Thu May 16 17:39:40 2019 +0200

    IMPALA-7369: part 2: Add INTERVAL expr support and built-in functions for DATE
    
    This change implements INTERVAL expression support for DATE type and
    adds several DATE related built-in functions. The efficiency of the
    DateValue::ToYearMonthDay() function used in many of the built-in
    functions below was also improved.
    
    The following functions are supported in Hive:
    
      INT YEAR(DATE d)
      Extracts year of the 'd' date, returns it as an int in 0-9999 range.
    
      INT MONTH(DATE d)
      Extracts month of the 'd' date and returns it as an int in 1-12
      range.
    
      INT DAY(DATE d), INT DAYOFMONTH(DATE d)
      Extracts day-of-month of the 'd' date and returns it as an int in
      1-31 range.
    
      INT QUARTER(DATE d)
      Extracts quarter of the 'd' date and returns it as an int in 1-4
      range.
    
      INT DAYOFWEEK(DATE d)
      Extracts day-of-week of the 'd' date and returns it as an int in
      1-7 range. 1 is Sunday and 7 is Saturday.
    
      INT DAYOFYEAR(DATE d)
      Extracts day-of-year of the 'd' date and returns it as an int in
      1-366 range.
    
      INT WEEKOFYEAR(DATE d)
      Extracts week-of-year of the 'd' date and returns it as an int in
      1-53 range.
    
      STRING DAYNAME(DATE d)
      Returns the day field from a 'd' date, converted to the string
      corresponding to that day name. The range of return values is
      "Sunday" to "Saturday".
    
      STRING MONTHNAME(DATE d)
      Returns the month field from a 'd' date, converted to the string
      corresponding to that month name. The range of return values is
      "January" to "December".
    
      DATE NEXT_DAY(DATE d, STRING weekday)
      Returns the first date which is later than 'd' and named as
      'weekday'. 'weekday' is 3 letters or full name of the day of the
      week.
    
      DATE LAST_DAY(DATE d)
      Returns the last day of the month which the 'd' date belongs to.
    
      INT DATEDIFF(DATE d1, DATE d2)
      Returns the number of days from 'd1' date to 'd2' date.
    
      DATE CURRENT_DATE()
      Returns the current date (in the local time zone).
    
      INT INT_MONTHS_BETWEEN(DATE d1, DATE d2)
      Returns the number of months between 'd1' and 'd2' dates, as an int
      representing only the full months that passed.
      If 'd1' represents an earlier date than 'd2', the result is
      negative.
    
      DOUBLE MONTHS_BETWEEN(DATE d1, DATE d2)
      Returns the number of months between 'd1' and 'd2' dates. Can
      include a fractional part representing extra days in addition to the
      full months between the dates. The fractional component is computed
      by dividing the difference in days by 31 (regardless of the month).
      If 'd1' represents an earlier date than 'd2', the result is
      negative.
    
      DATE ADD_YEARS(DATE d, INT/BIGINT num_years),
      DATE SUB_YEARS(DATE d, INT/BIGINT num_years)
      Adds/subtracts a specified number of years to a 'd' date value.
    
      DATE ADD_MONTHS(DATE d, INT/BIGINT num_months),
      DATE SUB_MONTHS(DATE d, INT/BIGINT num_months)
      Adds/subtracts a specified number of months to a date value.
      If 'd' is the last day of a month, the returned date will fall on
      the last day of the target month too.
    
      DATE ADD_DAYS(DATE d, INT/BIGINT num_days),
      DATE SUB_DAYS(DATE d, INT/BIGINT num_days)
      Adds/subtracts a specified number of days to a date value.
    
      DATE ADD_WEEKS(DATE d, INT/BIGINT num_weeks),
      DATE SUB_WEEKS(DATE d, INT/BIGINT num_weeks)
      Adds/subtracts a specified number of weeks to a date value.
    
    The following function doesn't exist in Hive but supported by Amazon
    Redshift
    
      INT DATE_CMP(DATE d1, DATE d2)
      Compares 'd1' and 'd2' dates. Returns:
      1. NULL, if either 'd1' or 'd2' is NULL
      2. -1 if d1 < d2
      3. 1 if d1 > d2
      4. 0 if d1 == d2
      (https://docs.aws.amazon.com/redshift/latest/dg/r_DATE_CMP.html)
    
    Change-Id: If404bffdaf055c769e79ffa8f193bac415cfdd1a
    Reviewed-on: http://gerrit.cloudera.org:8080/13648
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/date-benchmark.cc                | 129 +++++--
 be/src/codegen/impala-ir.cc                        |   1 +
 be/src/exprs/CMakeLists.txt                        |   1 +
 be/src/exprs/date-functions-ir.cc                  | 332 +++++++++++++++++
 be/src/exprs/date-functions.h                      | 160 +++++++++
 be/src/exprs/expr-test.cc                          | 394 ++++++++++++++++++++-
 be/src/exprs/scalar-expr-evaluator.cc              |   2 +
 be/src/exprs/timestamp-functions-ir.cc             |  31 +-
 be/src/exprs/timestamp-functions.cc                |  11 +
 be/src/exprs/timestamp-functions.h                 |  15 +-
 be/src/exprs/udf-builtins.cc                       |  33 +-
 be/src/runtime/date-test.cc                        | 330 +++++++++++++++--
 be/src/runtime/date-value.cc                       | 225 ++++++++++--
 be/src/runtime/date-value.h                        |  49 ++-
 common/function-registry/impala_functions.py       |  57 +++
 .../impala/analysis/TimestampArithmeticExpr.java   |  35 +-
 .../apache/impala/analysis/AnalyzeExprsTest.java   |  78 ++--
 17 files changed, 1705 insertions(+), 178 deletions(-)

diff --git a/be/src/benchmarks/date-benchmark.cc b/be/src/benchmarks/date-benchmark.cc
index c0f176c..574dcc7 100644
--- a/be/src/benchmarks/date-benchmark.cc
+++ b/be/src/benchmarks/date-benchmark.cc
@@ -15,7 +15,9 @@
 // specific language governing permissions and limitations
 // under the License.
 
+#include <iomanip>
 #include <iostream>
+#include <random>
 #include <vector>
 
 #include "cctz/civil_time.h"
@@ -26,59 +28,109 @@
 
 #include "common/names.h"
 
+using std::random_device;
+using std::mt19937;
+using std::uniform_int_distribution;
+
 using namespace impala;
 
 // Machine Info: Intel(R) Core(TM) i5-6600 CPU @ 3.30GHz
-// ToYear:             Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
+// ToYearMonthDay:     Function  iters/ms   10%ile   50%ile   90%ile     10%ile     50%ile     90%ile
 //                                                                   (relative) (relative) (relative)
 // --------------------------------------------------------------------------------------------------
-//           TestToYearMonthDay              0.167    0.169    0.169         1X         1X         1X
-//                   TestToYear               4.63     4.72     4.72      27.8X      27.8X      27.8X
+//       TestCctzToYearMonthDay               0.24    0.245     0.25         1X         1X         1X
+//           TestToYearMonthDay                1.3     1.33     1.38      5.42X      5.44X      5.53X
+//                   TestToYear               8.67     8.86     9.04      36.1X      36.1X      36.2X
+
+const cctz::civil_day EPOCH_DATE(1970, 1, 1);
 
 class TestData {
 public:
-  void AddRange(const DateValue& dv_min, const DateValue& dv_max) {
+  void AddRandomRange(const DateValue& dv_min, const DateValue& dv_max) {
     DCHECK(dv_min.IsValid());
     DCHECK(dv_max.IsValid());
-    DateValue dv = dv_min;
-    while (dv < dv_max) {
-      date_.push_back(dv);
-      dv = dv.AddDays(1);
+
+    int32_t min_dse, max_dse;
+    ignore_result(dv_min.ToDaysSinceEpoch(&min_dse));
+    ignore_result(dv_max.ToDaysSinceEpoch(&max_dse));
+
+    random_device rd;
+    mt19937 gen(rd());
+    // Random values in a [min_dse..max_dse] days range.
+    uniform_int_distribution<int32_t> dis_dse(min_dse, max_dse);
+
+    // Add random DateValue values in the [dv_min, dv_max] range.
+    for (int i = 0; i <= max_dse - min_dse; ++i) {
+      DateValue dv(dis_dse(gen));
       DCHECK(dv.IsValid());
+      date_.push_back(dv);
+    }
+    cctz_to_ymd_result_.resize(date_.size());
+    to_ymd_result_.resize(date_.size());
+    to_year_result_.resize(date_.size());
+  }
+
+  void CctzToYearMonthDay(const DateValue& dv, int* year, int* month, int* day) const {
+    int32_t days_since_epoch;
+    ignore_result(dv.ToDaysSinceEpoch(&days_since_epoch));
+
+    const cctz::civil_day cd = EPOCH_DATE + days_since_epoch;
+    *year = cd.year();
+    *month = cd.month();
+    *day = cd.day();
+  }
+
+  void TestCctzToYearMonthDay(int batch_size) {
+    DCHECK(date_.size() == cctz_to_ymd_result_.size());
+    for (int i = 0; i < batch_size; ++i) {
+      int n = date_.size();
+      for (int j = 0; j < n; ++j) {
+        CctzToYearMonthDay(date_[j],
+            &cctz_to_ymd_result_[j].year_,
+            &cctz_to_ymd_result_[j].month_,
+            &cctz_to_ymd_result_[j].day_);
+      }
     }
-    to_ymd_result_year_.resize(date_.size());
-    to_y_result_year_.resize(date_.size());
   }
 
   void TestToYearMonthDay(int batch_size) {
-    DCHECK(date_.size() == to_ymd_result_year_.size());
-    int month, day;
+    DCHECK(date_.size() == to_ymd_result_.size());
     for (int i = 0; i < batch_size; ++i) {
       int n = date_.size();
       for (int j = 0; j < n; ++j) {
-        ignore_result(date_[j].ToYearMonthDay(&to_ymd_result_year_[j], &month, &day));
+        ignore_result(date_[j].ToYearMonthDay(
+            &to_ymd_result_[j].year_,
+            &to_ymd_result_[j].month_,
+            &to_ymd_result_[j].day_));
       }
     }
   }
 
   void TestToYear(int batch_size) {
-    DCHECK(date_.size() == to_y_result_year_.size());
+    DCHECK(date_.size() == to_year_result_.size());
     for (int i = 0; i < batch_size; ++i) {
       int n = date_.size();
       for (int j = 0; j < n; ++j) {
-        ignore_result(date_[j].ToYear(&to_y_result_year_[j]));
+        ignore_result(date_[j].ToYear(&to_year_result_[j]));
       }
     }
   }
 
   bool CheckResults() {
-    DCHECK(to_ymd_result_year_.size() == to_y_result_year_.size());
+    DCHECK(to_ymd_result_.size() == cctz_to_ymd_result_.size());
+    DCHECK(to_year_result_.size() == cctz_to_ymd_result_.size());
     bool ok = true;
-    for (int i = 0; i < to_ymd_result_year_.size(); ++i) {
-      if (to_ymd_result_year_[i] != to_y_result_year_[i]) {
-        cerr << "Incorrect results " << date_[i] << ": "
-            << to_ymd_result_year_[i] << " != " << to_y_result_year_[i]
-            << endl;
+    for (int i = 0; i < cctz_to_ymd_result_.size(); ++i) {
+      if (to_ymd_result_[i] != cctz_to_ymd_result_[i]) {
+        cerr << "Incorrect results (ToYearMonthDay() vs CctzToYearMonthDay()): "
+             << date_[i] << ": " << to_ymd_result_[i].ToString() << " != "
+             << cctz_to_ymd_result_[i].ToString() << endl;
+        ok = false;
+      }
+      if (to_year_result_[i] != cctz_to_ymd_result_[i].year_) {
+        cerr << "Incorrect results (ToYear() vs CctzToYearMonthDay()): " << date_[i]
+             << ": " << to_year_result_[i] << " != " << cctz_to_ymd_result_[i].year_
+             << endl;
         ok = false;
       }
     }
@@ -86,11 +138,34 @@ public:
   }
 
 private:
+  struct YearMonthDayResult {
+    int year_;
+    int month_;
+    int day_;
+
+    string ToString() const {
+      stringstream ss;
+      ss << std::setfill('0') << setw(4) << year_ << "-" << setw(2) << month_ << "-"
+         << setw(2) << day_;
+      return ss.str();
+    }
+
+    bool operator!=(const YearMonthDayResult& other) const {
+      return year_ != other.year_ || month_ != other.month_ || day_ != other.day_;
+    }
+  };
+
   vector<DateValue> date_;
-  vector<int> to_ymd_result_year_;
-  vector<int> to_y_result_year_;
+  vector<YearMonthDayResult> cctz_to_ymd_result_;
+  vector<YearMonthDayResult> to_ymd_result_;
+  vector<int> to_year_result_;
 };
 
+void TestCctzToYearMonthDay(int batch_size, void* d) {
+  TestData* data = reinterpret_cast<TestData*>(d);
+  data->TestCctzToYearMonthDay(batch_size);
+}
+
 void TestToYearMonthDay(int batch_size, void* d) {
   TestData* data = reinterpret_cast<TestData*>(d);
   data->TestToYearMonthDay(batch_size);
@@ -106,10 +181,12 @@ int main(int argc, char* argv[]) {
   cout << Benchmark::GetMachineInfo() << endl;
 
   TestData data;
-  data.AddRange(DateValue(1965, 1, 1), DateValue(2020, 12, 31));
+  data.AddRandomRange(DateValue(1965, 1, 1), DateValue(2020, 12, 31));
 
-  // Benchmark DateValue::ToYearMonthDay() vs DateValue::ToYear()
-  Benchmark suite("ToYear");
+  // Benchmark TestData::CctzToYearMonthDay(), DateValue::ToYearMonthDay() and
+  // DateValue::ToYear().
+  Benchmark suite("ToYearMonthDay");
+  suite.AddBenchmark("TestCctzToYearMonthDay", TestCctzToYearMonthDay, &data);
   suite.AddBenchmark("TestToYearMonthDay", TestToYearMonthDay, &data);
   suite.AddBenchmark("TestToYear", TestToYear, &data);
   cout << suite.Measure();
diff --git a/be/src/codegen/impala-ir.cc b/be/src/codegen/impala-ir.cc
index b07d9c2..1459316 100644
--- a/be/src/codegen/impala-ir.cc
+++ b/be/src/codegen/impala-ir.cc
@@ -43,6 +43,7 @@
 #include "exprs/cast-functions-ir.cc"
 #include "exprs/compound-predicates-ir.cc"
 #include "exprs/conditional-functions-ir.cc"
+#include "exprs/date-functions-ir.cc"
 #include "exprs/decimal-functions-ir.cc"
 #include "exprs/decimal-operators-ir.cc"
 #include "exprs/in-predicate-ir.cc"
diff --git a/be/src/exprs/CMakeLists.txt b/be/src/exprs/CMakeLists.txt
index 734892d..9863182 100644
--- a/be/src/exprs/CMakeLists.txt
+++ b/be/src/exprs/CMakeLists.txt
@@ -35,6 +35,7 @@ add_library(Exprs
   compound-predicates-ir.cc
   conditional-functions.cc
   conditional-functions-ir.cc
+  date-functions-ir.cc
   decimal-functions-ir.cc
   decimal-operators-ir.cc
   expr.cc
diff --git a/be/src/exprs/date-functions-ir.cc b/be/src/exprs/date-functions-ir.cc
new file mode 100644
index 0000000..0e235c4
--- /dev/null
+++ b/be/src/exprs/date-functions-ir.cc
@@ -0,0 +1,332 @@
+// 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.
+
+#include "exprs/date-functions.h"
+
+#include <boost/date_time/posix_time/posix_time.hpp>
+#include <boost/date_time/local_time/local_time.hpp>
+
+#include "cctz/civil_time.h"
+#include "exprs/anyval-util.h"
+#include "exprs/timestamp-functions.h"
+#include "exprs/udf-builtins.h"
+#include "runtime/datetime-parse-util.h"
+#include "runtime/timestamp-value.h"
+#include "runtime/timestamp-value.inline.h"
+#include "udf/udf.h"
+#include "udf/udf-internal.h"
+
+#include "common/names.h"
+
+namespace impala {
+
+using datetime_parse_util::DateTimeFormatContext;
+using datetime_parse_util::ParseFormatTokens;
+
+IntVal DateFunctions::Year(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  int year;
+  if (!dv.ToYear(&year)) return IntVal::null();
+  return IntVal(year);
+}
+
+IntVal DateFunctions::Quarter(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  int year, month, day;
+  if (!dv.ToYearMonthDay(&year, &month, &day)) return IntVal::null();
+  return IntVal((month - 1) / 3 + 1);
+}
+
+IntVal DateFunctions::Month(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  int year, month, day;
+  if (!dv.ToYearMonthDay(&year, &month, &day)) return IntVal::null();
+  return IntVal(month);
+}
+
+IntVal DateFunctions::DayOfWeek(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  // DAYOFWEEK(DATE) sql function returns day-of-week in [1, 7] range, where 1 = Sunday.
+  // dv.WeekDay() returns day-of-week in [0, 6] range. 0 = Monday and 6 = Sunday.
+  int wday = dv.WeekDay();
+  if (wday == -1) return IntVal::null();
+  return IntVal((wday + 1) % 7 + 1);
+}
+
+IntVal DateFunctions::DayOfMonth(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  int year, month, day;
+  if (!dv.ToYearMonthDay(&year, &month, &day)) return IntVal::null();
+  return IntVal(day);
+}
+
+IntVal DateFunctions::DayOfYear(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  // Get the day of the year. DAYOFYEAR(DATE) sql function returns day-of-year in
+  // [1, 366]  range.
+  int yday = dv.DayOfYear();
+  if (yday == -1) return IntVal::null();
+  return IntVal(yday);
+}
+
+IntVal DateFunctions::WeekOfYear(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return IntVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  int yweek = dv.WeekOfYear();
+  if (yweek == -1) return IntVal::null();
+  return IntVal(yweek);
+}
+
+StringVal DateFunctions::LongDayName(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return StringVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  // CCTZ has 0 = Monday and 6 = Sunday.
+  int wday = dv.WeekDay();
+  if (wday == -1) return StringVal::null();
+
+  DCHECK_GE(wday, 0);
+  DCHECK_LE(wday, 6);
+  wday = (wday + 1) % 7;
+  const string& day_name = TimestampFunctions::DAYNAME_ARRAY[wday];
+  return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(day_name.data())),
+      day_name.size());
+}
+
+StringVal DateFunctions::LongMonthName(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return StringVal::null();
+  DateValue dv = DateValue::FromDateVal(d_val);
+
+  int year, month, day;
+  if (!dv.ToYearMonthDay(&year, &month, &day)) return StringVal::null();
+
+  DCHECK_GE(month, 1);
+  DCHECK_LE(month, 12);
+  const string& mn = TimestampFunctions::MONTHNAME_ARRAY[month - 1];
+  return StringVal(reinterpret_cast<uint8_t*>(const_cast<char*>(mn.data())), mn.size());
+}
+
+DateVal DateFunctions::NextDay(FunctionContext* context, const DateVal& d_val,
+    const StringVal& weekday) {
+  if (weekday.is_null) {
+    context->SetError("Invalid Day: NULL");
+    return DateVal::null();
+  }
+
+  StringVal lweekday = UdfBuiltins::Lower(context, weekday);
+  string weekday_str = string(reinterpret_cast<const char*>(lweekday.ptr),
+      lweekday.len);
+
+  // DAYNAME_MAP maps Sunday to 0 and Saturday to 6
+  const auto it = TimestampFunctions::DAYNAME_MAP.find(weekday_str);
+  if (it == TimestampFunctions::DAYNAME_MAP.end()) {
+    context->SetError(Substitute("Invalid Day: $0", weekday_str).c_str());
+    return DateVal::null();
+  } else {
+    if (d_val.is_null) return DateVal::null();
+    DateValue dv = DateValue::FromDateVal(d_val);
+
+    // WeekDay() returns 0 for Monday and 6 for Sunday.
+    int wday = dv.WeekDay();
+    if (wday == -1) return DateVal::null();
+    DCHECK_GE(wday, 0);
+    DCHECK_LE(wday, 6);
+
+    int delta_days = it->second - (wday + 1) % 7;
+    delta_days = delta_days <= 0 ? delta_days + 7 : delta_days;
+    DCHECK_GE(delta_days, 1);
+    DCHECK_LE(delta_days, 7);
+
+    return dv.AddDays(delta_days).ToDateVal();
+  }
+}
+
+DateVal DateFunctions::LastDay(FunctionContext* context, const DateVal& d_val) {
+  if (d_val.is_null) return DateVal::null();
+  return DateValue::FromDateVal(d_val).LastDay().ToDateVal();
+}
+
+IntVal DateFunctions::DateDiff(FunctionContext* context, const DateVal& d_val1,
+    const DateVal& d_val2) {
+  if (d_val1.is_null || d_val2.is_null) return IntVal::null();
+  DateValue dv1 = DateValue::FromDateVal(d_val1);
+  DateValue dv2 = DateValue::FromDateVal(d_val2);
+
+  int32_t dse1, dse2;
+  if (!dv1.ToDaysSinceEpoch(&dse1) || !dv2.ToDaysSinceEpoch(&dse2)) return IntVal::null();
+  return IntVal(dse1 - dse2);
+}
+
+DateVal DateFunctions::CurrentDate(FunctionContext* context) {
+  const TimestampValue* now = context->impl()->state()->now();
+  const boost::gregorian::date& d = now->date();
+  return DateValue(d.year(), d.month(), d.day()).ToDateVal();
+}
+
+IntVal DateFunctions::DateCmp(FunctionContext* context, const DateVal& d_val1,
+    const DateVal& d_val2) {
+  if (d_val1.is_null || d_val2.is_null) return IntVal::null();
+  if (d_val1.val > d_val2.val) return 1;
+  if (d_val1.val < d_val2.val) return -1;
+  return 0;
+}
+
+IntVal DateFunctions::IntMonthsBetween(FunctionContext* context,
+    const DateVal& d_val1, const DateVal& d_val2) {
+  DoubleVal months_between = MonthsBetween(context, d_val1, d_val2);
+  if (months_between.is_null) return IntVal::null();
+  return IntVal(static_cast<int32_t>(months_between.val));
+}
+
+DoubleVal DateFunctions::MonthsBetween(FunctionContext* context,
+    const DateVal& d_val1, const DateVal& d_val2) {
+  if (d_val1.is_null || d_val2.is_null) return DoubleVal::null();
+  DateValue dv1 = DateValue::FromDateVal(d_val1);
+  DateValue dv2 = DateValue::FromDateVal(d_val2);
+
+  double months_between;
+  if (!dv1.MonthsBetween(dv2, &months_between)) return DoubleVal::null();
+  return DoubleVal(months_between);
+}
+
+template <bool is_add, typename AnyIntVal>
+DateVal DateFunctions::AddSubYears(FunctionContext* context, const DateVal& d_val,
+    const AnyIntVal& num_years) {
+  if (d_val.is_null || num_years.is_null) return DateVal::null();
+
+  const DateValue dv = DateValue::FromDateVal(d_val).AddYears(
+      is_add ? num_years.val : -num_years.val);
+  return dv.ToDateVal();
+}
+
+template <bool is_add, typename AnyIntVal, bool keep_last_day>
+DateVal DateFunctions::AddSubMonths(FunctionContext* context, const DateVal& d_val,
+    const AnyIntVal& num_months) {
+  if (d_val.is_null || num_months.is_null) return DateVal::null();
+
+  const DateValue dv = DateValue::FromDateVal(d_val).AddMonths(
+      is_add ? num_months.val : -num_months.val, keep_last_day);
+  return dv.ToDateVal();
+}
+
+template <bool is_add, typename AnyIntVal>
+DateVal DateFunctions::AddSubDays(FunctionContext* context, const DateVal& d_val,
+    const AnyIntVal& num_days) {
+  if (d_val.is_null || num_days.is_null) return DateVal::null();
+
+  const DateValue dv = DateValue::FromDateVal(d_val).AddDays(
+      is_add ? num_days.val : -num_days.val);
+  return dv.ToDateVal();
+}
+
+template <bool is_add, typename AnyIntVal>
+DateVal DateFunctions::AddSubWeeks(FunctionContext* context, const DateVal& d_val,
+    const AnyIntVal& num_weeks) {
+  if (d_val.is_null || num_weeks.is_null) return DateVal::null();
+
+  // Sanity check: make sure that the number of weeks converted to days fits into 64-bits.
+  int64_t weeks = is_add ? num_weeks.val : -num_weeks.val;
+  if (weeks > numeric_limits<int64_t>::max() / 7
+      || weeks < numeric_limits<int64_t>::min() / 7) {
+    return DateVal::null();
+  }
+
+  const DateValue dv = DateValue::FromDateVal(d_val).AddDays(weeks * 7);
+  return dv.ToDateVal();
+}
+
+// Explicit template instantiation is required for proper linking. These functions
+// are only indirectly called via a function pointer provided by the opcode registry
+// which does not trigger implicit template instantiation.
+// Must be kept in sync with common/function-registry/impala_functions.py.
+template DateVal
+DateFunctions::AddSubYears<true, IntVal>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubYears<true, BigIntVal>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+template DateVal
+DateFunctions::AddSubYears<false, IntVal>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubYears<false, BigIntVal>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+
+template DateVal
+DateFunctions::AddSubMonths<true, IntVal, true>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<true, IntVal, false>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<true, BigIntVal, true>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<true, BigIntVal, false>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<false, IntVal, true>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<false, IntVal, false>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<false, BigIntVal, true>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+template DateVal
+DateFunctions::AddSubMonths<false, BigIntVal, false>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+
+template DateVal
+DateFunctions::AddSubDays<true, IntVal>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubDays<true, BigIntVal>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+template DateVal
+DateFunctions::AddSubDays<false, IntVal>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubDays<false, BigIntVal>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+
+template DateVal
+DateFunctions::AddSubWeeks<true, IntVal>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubWeeks<true, BigIntVal>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+template DateVal
+DateFunctions::AddSubWeeks<false, IntVal>(FunctionContext* context,
+    const DateVal& d_val, const IntVal& count);
+template DateVal
+DateFunctions::AddSubWeeks<false, BigIntVal>(FunctionContext* context,
+    const DateVal& d_val, const BigIntVal& count);
+}
diff --git a/be/src/exprs/date-functions.h b/be/src/exprs/date-functions.h
new file mode 100644
index 0000000..0f9fcce
--- /dev/null
+++ b/be/src/exprs/date-functions.h
@@ -0,0 +1,160 @@
+// 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.
+
+
+#pragma once
+
+#include <string>
+#include <unordered_map>
+
+#include "cctz/civil_time.h"
+#include "common/status.h"
+#include "udf/udf.h"
+
+namespace impala {
+
+using impala_udf::FunctionContext;
+using impala_udf::AnyVal;
+using impala_udf::BooleanVal;
+using impala_udf::TinyIntVal;
+using impala_udf::SmallIntVal;
+using impala_udf::IntVal;
+using impala_udf::BigIntVal;
+using impala_udf::FloatVal;
+using impala_udf::DoubleVal;
+using impala_udf::TimestampVal;
+using impala_udf::StringVal;
+using impala_udf::DecimalVal;
+using impala_udf::DateVal;
+
+class DateFunctions {
+ public:
+  /// YEAR(DATE d)
+  /// Extracts year of the 'd_val' date, returns it as an int in 0-9999 range
+  static IntVal Year(FunctionContext* context, const DateVal& d_val);
+
+  /// MONTH(DATE d)
+  /// Extracts month of the 'd_val' date and returns it as an int in 1-12 range.
+  static IntVal Month(FunctionContext* context, const DateVal& d_val);
+
+  /// DAY(DATE d), DAYOFMONTH(DATE d)
+  /// Extracts day-of-month of the 'd_val' date and returns it as an int in 1-31 range.
+  static IntVal DayOfMonth(FunctionContext* context, const DateVal& d_val);
+
+  /// QUARTER(DATE d)
+  /// Extracts quarter of the 'd_val' date and returns it as an int in 1-4 range.
+  static IntVal Quarter(FunctionContext* context, const DateVal& d_val);
+
+  /// DAYOFWEEK(DATE d)
+  /// Extracts day-of-week of the 'd_val' date and returns it as an int in 1-7 range.
+  /// 1 is Sunday and 7 is Saturday.
+  static IntVal DayOfWeek(FunctionContext* context, const DateVal& d_val);
+
+  /// DAYOFYEAR(DATE d)
+  /// Extracts day-of-year of the 'd_val' date and returns it as an int in 1-366 range.
+  static IntVal DayOfYear(FunctionContext* context, const DateVal& d_val);
+
+  /// WEEKOFYEAR(DATE d)
+  /// Extracts week-of-year of the 'd_val' date and returns it as an int in 1-53 range.
+  static IntVal WeekOfYear(FunctionContext* context, const DateVal& d_val);
+
+  /// DAYNAME(DATE d)
+  /// Returns the day field from a 'd_val' date, converted to the string corresponding to
+  /// that day name. The range of return values is "Sunday" to "Saturday".
+  static StringVal LongDayName(FunctionContext* context, const DateVal& d_val);
+
+  /// MONTHNAME(DATE d)
+  /// Returns the month field from a 'd_val' date, converted to the string corresponding
+  /// to that month name. The range of return values is "January" to "December".
+  static StringVal LongMonthName(FunctionContext* context, const DateVal& d_val);
+
+  /// NEXT_DAY(DATE d, STRING weekday)
+  /// Returns the first date which is later than 'd_val' and named as 'weekday'.
+  /// 'weekday' is 3 letters or full name of the day of the week.
+  static DateVal NextDay(FunctionContext* context, const DateVal& d_val,
+      const StringVal& weekday);
+
+  /// LAST_DAY(DATE d)
+  /// Returns the last day of the month which the 'd_val' date belongs to.
+  static DateVal LastDay(FunctionContext* context, const DateVal& d_val);
+
+  /// DATEDIFF(DATE d1, DATE d2)
+  /// Returns the number of days from 'd_val1' date to 'd_val2' date.
+  static IntVal DateDiff(FunctionContext* context, const DateVal& d_val1,
+      const DateVal& d_val2);
+
+  /// CURRENT_DATE()
+  /// Returns the current date (in the local time zone).
+  static DateVal CurrentDate(FunctionContext* context);
+
+  /// DATE_CMP(DATE d1, DATE d2)
+  /// Compares 'd_val1' and 'd_val2' dates. Returns:
+  /// 1. null, if either 'd_val1' or 'd_val2' is null
+  /// 2. -1 if d_val1 < d_val2
+  /// 3. 1 if d_val1 > d_val2
+  /// 4. 1 if d_val1 == d_val2
+  static IntVal DateCmp(FunctionContext* context, const DateVal& d_val1,
+      const DateVal& d_val2);
+
+  /// INT_MONTHS_BETWEEN(DATE d1, DATE d2)
+  /// Returns the number of months between 'd_val1' and 'd_val2' dates, as an int
+  /// representing only the full months that passed.
+  /// If 'd_val1' represents an earlier date than 'd_val2', the result is negative.
+  static IntVal IntMonthsBetween(FunctionContext* context, const DateVal& d_val1,
+      const DateVal& d_val2);
+
+  /// MONTHS_BETWEEN(DATE d1, DATE d2)
+  /// Returns the number of months between 'd_val1' and 'd_val2' dates. Can include a
+  /// fractional part representing extra days in addition to the full months between the
+  /// dates. The fractional component is computed by dividing the difference in days by 31
+  /// (regardless of the month).
+  /// If 'd_val1' represents an earlier date than 'd_val2', the result is negative.
+  static DoubleVal MonthsBetween(FunctionContext* context, const DateVal& d_val1,
+      const DateVal& d_val2);
+
+  /// ADD_YEARS(DATE d, INT num_years), ADD_YEARS(DATE d, BIGINT num_years)
+  /// SUB_YEARS(DATE d, INT num_years), SUB_YEARS(DATE d, BIGINT num_years)
+  /// Adds/subtracts a specified number of years to a date value.
+  template <bool is_add, typename AnyIntVal>
+  static DateVal AddSubYears(FunctionContext* context, const DateVal& d_val,
+      const AnyIntVal& num_years);
+
+  /// ADD_MONTHS(DATE d, INT num_months), ADD_MONTHS(DATE d, BIGINT num_months)
+  /// SUB_MONTHS(DATE d, INT num_months), SUB_MONTHS(DATE d, BIGINT num_months)
+  /// Adds/subtracts a specified number of months to a date value.
+  /// If 'keep_last_day' is set and 'd_val' is the last day of a month, the returned date
+  /// will fall on the last day of the target month too.
+  template <bool is_add, typename AnyIntVal, bool keep_last_day>
+  static DateVal AddSubMonths(FunctionContext* context, const DateVal& d_val,
+      const AnyIntVal& num_months);
+
+  /// ADD_DAYS(DATE d, INT num_days), ADD_DAYS(DATE d, BIGINT num_days)
+  /// SUB_DAYS(DATE d, INT num_days), SUB_DAYS(DATE d, BIGINT num_days)
+  /// Adds/subtracts a specified number of days to a date value.
+  template <bool is_add, typename AnyIntVal>
+  static DateVal AddSubDays(FunctionContext* context, const DateVal& d_val,
+      const AnyIntVal& num_days);
+
+  /// ADD_WEEKS(DATE d, INT num_weeks), ADD_WEEKS(DATE d, BIGINT num_weeks)
+  /// SUB_WEEKS(DATE d, INT num_weeks), SUB_WEEKS(DATE d, BIGINT num_weeks)
+  /// Adds/subtracts a specified number of weeks to a date value.
+  template <bool is_add, typename AnyIntVal>
+  static DateVal AddSubWeeks(FunctionContext* context, const DateVal& d_val,
+      const AnyIntVal& num_weeks);
+};
+
+} // namespace impala
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index f89d953..76fdf1a 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -504,6 +504,15 @@ class ExprTest : public testing::TestWithParam<std::tuple<bool, bool>> {
     TestIsNull("next_day('000000000000000','wed')", TYPE_TIMESTAMP);
     TestIsNull("next_day('hell world!','fRiDaY')", TYPE_TIMESTAMP);
     TestIsNull("next_day('t1c7t0c9','sunDAY')", TYPE_TIMESTAMP);
+    TestIsNull("next_day(NULL ,'sunDAY')", TYPE_TIMESTAMP);
+
+    // Invalid input: wrong weekday parameter
+    for (const string& day: { "s", "SA", "satu", "not-a-day" }) {
+      const string expr = "next_day('2013-12-25','" + day + "')";
+      TestError(expr);
+    }
+    TestError("next_day('2013-12-25', NULL)");
+    TestError("next_day(NULL, NULL)");
   }
 
 // This macro adds a scoped trace to provide the line number of the caller upon failure.
@@ -5175,6 +5184,7 @@ TEST_P(ExprTest, UtilityFunctions) {
   TestStringValue("typeOf(utc_timestamp())", "TIMESTAMP");
   TestStringValue("typeOf(DATE '2011-01-01')", "DATE");
   TestStringValue("typeOf(cast(now() as DATE))", "DATE");
+  TestStringValue("typeOf(current_date())", "DATE");
   TestStringValue("typeOf(cast(10 as DECIMAL))", "DECIMAL(9,0)");
   TestStringValue("typeOf(0.0)", "DECIMAL(1,1)");
   TestStringValue("typeOf(3.14)", "DECIMAL(3,2)");
@@ -7424,7 +7434,7 @@ TEST_P(ExprTest, TimestampFunctions) {
 TEST_P(ExprTest, TruncForDateTest) {
   // trunc(date, string unit)
   // Truncate date to year
-  for (const string unit: { "SYYYY", "YYYY", "YEAR", "SYEAR", "YYY", "YY", "Y" }) {
+  for (const string& unit: { "SYYYY", "YYYY", "YEAR", "SYEAR", "YYY", "YY", "Y" }) {
     const string expr = "trunc(date'2014-04-01', '" + unit + "')";
     TestDateValue(expr, DateValue(2014, 1, 1));
   }
@@ -7521,9 +7531,11 @@ TEST_P(ExprTest, TruncForDateTest) {
     const string expr = "trunc(date'2012-09-10', '" + unit + "')";
     TestNonOkStatus(expr);  // Invalid Truncate Unit
   }
+  TestNonOkStatus("trunc(date'2012-09-10', NULL)");  // Invalid Truncate Unit
+  TestNonOkStatus("trunc(cast(NULL as date), NULL)");  // Invalid Truncate Unit
 
+  // Truncating NULL date returns NULL.
   TestIsNull("trunc(cast(NULL as date), 'DDD')", TYPE_DATE);
-  TestNonOkStatus("trunc(cast(NULL as date), NULL)");
 }
 
 TEST_P(ExprTest, DateTruncForDateTest) {
@@ -7592,9 +7604,11 @@ TEST_P(ExprTest, DateTruncForDateTest) {
     const string expr = "date_trunc('" + unit + "', date '2012-09-10')";
     TestNonOkStatus(expr);  // Invalid Date Truncate Unit
   }
+  TestNonOkStatus("date_trunc(NULL, date '2012-09-10'");  // Invalid Date Truncate Unit
+  TestNonOkStatus("date_trunc(NULL, cast(NULL as date))");  // Invalid Date Truncate Unit
 
+  // Truncating NULL date returns NULL.
   TestIsNull("date_trunc('DAY', cast(NULL as date))", TYPE_DATE);
-  TestNonOkStatus("date_trunc(NULL, cast(NULL as date))");  // Invalid Date Truncate Unit
 }
 
 TEST_P(ExprTest, ExtractAndDatePartForDateTest) {
@@ -7633,10 +7647,11 @@ TEST_P(ExprTest, ExtractAndDatePartForDateTest) {
     const string expr = "extract(date '2012-09-10', '" + field + "')";
     TestNonOkStatus(expr);  // Invalid Extract Field
   }
+  TestNonOkStatus("extract(date '2012-09-10', NULL)");  // Invalid Extract Field
+  TestNonOkStatus("extract(cast(NULL as date), NULL)");  // Invalid Extract Field
 
   TestIsNull("extract(cast(NULL as date), 'YEAR')", TYPE_BIGINT);
   TestIsNull("extract(YEAR from cast(NULL as date))", TYPE_BIGINT);
-  TestNonOkStatus("extract(cast(NULL as date), NULL)");
 
   // date_part, same as extract function but with arguments swapped
   TestValue("date_part('YEAR', date '2006-05-12')", TYPE_BIGINT, 2006);
@@ -7668,9 +7683,378 @@ TEST_P(ExprTest, ExtractAndDatePartForDateTest) {
     const string expr = "date_part('" + field + "', date '2012-09-10')";
     TestNonOkStatus(expr);  // Invalid Date Part Field
   }
+  TestNonOkStatus("date_part(MULL, date '2012-09-10')");  // Invalid Date Part Field
+  TestNonOkStatus("date_part(MULL, cast(NULL as date))");  // Invalid Date Part Field
 
   TestIsNull("date_part('YEAR', cast(NULL as date))", TYPE_BIGINT);
-  TestNonOkStatus("date_part(MULL, cast(NULL as date))");  // Invalid Date Part Field
+}
+
+TEST_P(ExprTest, DateFunctions) {
+  // year:
+  TestValue("year(date '2019-06-05')", TYPE_INT, 2019);
+  TestValue("year(date '9999-12-31')", TYPE_INT, 9999);
+  TestValue("year(date '0000-01-01')", TYPE_INT, 0);
+  TestIsNull("year(cast(NULL as date))", TYPE_INT);
+
+  // Test that the name-resolution algorithm picks up the TIMESTAMP-version of year() if
+  // year() is called with a STRING.
+  TestValue("year('2019-06-05')", TYPE_INT, 2019);
+  // 1399-12-31 is out of the valid TIMESTAMP range, year(TIMESTAMP) returns NULL.
+  TestIsNull("year('1399-12-31')", TYPE_INT);
+  // year(DATE) returns the correct result.
+  TestValue("year(DATE '1399-12-31')", TYPE_INT, 1399);
+  // Test that calling year(TIMESTAMP) with an invalid argument returns NULL.
+  TestIsNull("year('2019-02-29')", TYPE_INT);
+  // Test that calling year(DATE) with an invalid argument returns an error.
+  TestError("year(DATE '2019-02-29')");
+
+  // month:
+  TestValue("month(date '2019-06-05')", TYPE_INT, 6);
+  TestValue("month(date '9999-12-31')", TYPE_INT, 12);
+  TestValue("month(date '0000-01-01')", TYPE_INT, 1);
+  TestIsNull("month(cast(NULL as date))", TYPE_INT);
+
+  // monthname:
+  TestStringValue("monthname(date '2019-06-05')", "June");
+  TestStringValue("monthname(date '9999-12-31')", "December");
+  TestStringValue("monthname(date '0000-01-01')", "January");
+  TestIsNull("monthname(cast(NULL as date))", TYPE_STRING);
+
+  // day, dayofmonth:
+  TestValue("day(date '2019-06-05')", TYPE_INT, 5);
+  TestValue("day(date '9999-12-31')", TYPE_INT, 31);
+  TestValue("day(date '0000-01-01')", TYPE_INT, 1);
+  TestIsNull("day(cast(NULL as date))", TYPE_INT);
+  TestValue("dayofmonth(date '2019-06-07')", TYPE_INT, 7);
+  TestValue("dayofmonth(date '9999-12-31')", TYPE_INT, 31);
+  TestValue("dayofmonth(date '0000-01-01')", TYPE_INT, 1);
+  TestIsNull("dayofmonth(cast(NULL as date))", TYPE_INT);
+
+  // quarter:
+  TestValue("quarter(date '2019-01-01')", TYPE_INT, 1);
+  TestValue("quarter(date '2019-03-31')", TYPE_INT, 1);
+  TestValue("quarter(date '2019-04-01')", TYPE_INT, 2);
+  TestValue("quarter(date '2019-06-30')", TYPE_INT, 2);
+  TestValue("quarter(date '2019-07-01')", TYPE_INT, 3);
+  TestValue("quarter(date '2019-09-30')", TYPE_INT, 3);
+  TestValue("quarter(date '2019-10-01')", TYPE_INT, 4);
+  TestValue("quarter(date '2019-12-31')", TYPE_INT, 4);
+  TestValue("quarter(date '9999-12-31')", TYPE_INT, 4);
+  TestValue("quarter(date '0000-01-01')", TYPE_INT, 1);
+  TestIsNull("quarter(cast(NULL as date))", TYPE_INT);
+
+  // dayofweek:
+  TestValue("dayofweek(date '2019-06-05')", TYPE_INT, 4);
+  // 9999-12-31 is Friday.
+  TestValue("dayofweek(date '9999-12-31')", TYPE_INT, 6);
+  // 0000-01-01 is Saturday.
+  TestValue("dayofweek(date '0000-01-01')", TYPE_INT, 7);
+  TestIsNull("dayofweek(cast(NULL as date))", TYPE_INT);
+
+  // dayname:
+  TestStringValue("dayname(date '2019-06-03')", "Monday");
+  TestStringValue("dayname(date '2019-06-04')", "Tuesday");
+  TestStringValue("dayname(date '2019-06-05')", "Wednesday");
+  TestStringValue("dayname(date '2019-06-06')", "Thursday");
+  TestStringValue("dayname(date '2019-06-07')", "Friday");
+  TestStringValue("dayname(date '2019-06-08')", "Saturday");
+  TestStringValue("dayname(date '2019-06-09')", "Sunday");
+  TestStringValue("dayname(date '9999-12-31')", "Friday");
+  TestStringValue("dayname(date '0000-01-01')", "Saturday");
+  TestIsNull("dayname(cast(NULL as date))", TYPE_STRING);
+
+  // dayofyear:
+  TestValue("dayofyear(date '2019-01-01')", TYPE_INT, 1);
+  TestValue("dayofyear(date '2019-12-31')", TYPE_INT, 365);
+  TestValue("dayofyear(date '2019-06-05')", TYPE_INT, 31 + 28 + 31 + 30 + 31 + 5);
+  TestValue("dayofyear(date '2016-12-31')", TYPE_INT, 366);
+  TestValue("dayofyear(date '2016-06-05')", TYPE_INT, 31 + 29 + 31 + 30 + 31 + 5);
+  TestValue("dayofyear(date '9999-12-31')", TYPE_INT, 365);
+  TestValue("dayofyear(date '0000-01-01')", TYPE_INT, 1);
+  TestIsNull("dayofyear(cast(NULL as date))", TYPE_INT);
+
+  // week, weekofyear
+  // 2019-01-01 is Tuesday, it belongs to the first week of the year.
+  TestValue("weekofyear(date '2018-12-31')", TYPE_INT, 1);
+  TestValue("weekofyear(date '2019-01-01')", TYPE_INT, 1);
+  TestValue("weekofyear(date '2019-01-06')", TYPE_INT, 1);
+  TestValue("weekofyear(date '2019-01-07')", TYPE_INT, 2);
+  TestValue("weekofyear(date '2019-06-05')", TYPE_INT, 23);
+  TestValue("weekofyear(date '2019-12-23')", TYPE_INT, 52);
+  TestValue("weekofyear(date '2019-12-29')", TYPE_INT, 52);
+  TestValue("weekofyear(date '2019-12-30')", TYPE_INT, 1);
+  // Year 2015 has 53 weeks. 2015-12-31 is Thursday.
+  TestValue("weekofyear(date '2015-12-31')", TYPE_INT, 53);
+  TestValue("week(date '2018-12-31')", TYPE_INT, 1);
+  TestValue("week(date '2019-01-01')", TYPE_INT, 1);
+  TestValue("week(date '2019-01-06')", TYPE_INT, 1);
+  TestValue("week(date '2019-01-07')", TYPE_INT, 2);
+  TestValue("week(date '2019-06-05')", TYPE_INT, 23);
+  TestValue("week(date '2019-12-23')", TYPE_INT, 52);
+  TestValue("week(date '2019-12-29')", TYPE_INT, 52);
+  TestValue("week(date '2019-12-30')", TYPE_INT, 1);
+  TestValue("week(date '2015-12-31')", TYPE_INT, 53);
+  // 0000-01-01 is Saturday. It belongs to the last week of the previous year.
+  TestValue("weekofyear(date '0000-01-01')", TYPE_INT, 52);
+  TestValue("week(date '0000-01-01')", TYPE_INT, 52);
+  // 9999-12-31 is Friday. It belongs to the last week of the year.
+  TestValue("weekofyear(date '9999-12-31')", TYPE_INT, 52);
+  TestValue("week(date '9999-12-31')", TYPE_INT, 52);
+  TestIsNull("weekofyear(cast(NULL as date))", TYPE_INT);
+  TestIsNull("week(cast(NULL as date))", TYPE_INT);
+
+  // next_day:
+  // 2019-06-05 is Wednesday.
+  TestDateValue("next_day(date '2019-06-05', 'monday')", DateValue(2019, 6, 10));
+  TestDateValue("next_day(date '2019-06-05', 'TUE')", DateValue(2019, 6, 11));
+  TestDateValue("next_day(date '2019-06-05', 'Wed')", DateValue(2019, 6, 12));
+  TestDateValue("next_day(date '2019-06-05', 'THursdaY')", DateValue(2019, 6, 6));
+  TestDateValue("next_day(date '2019-06-05', 'fRI')", DateValue(2019, 6, 7));
+  TestDateValue("next_day(date '2019-06-05', 'saturDAY')", DateValue(2019, 6, 8));
+  TestDateValue("next_day(date '2019-06-05', 'suN')", DateValue(2019, 6, 9));
+  // 0000-01-01 is Saturday
+  TestDateValue("next_day(date '0000-01-01', 'SAT')", DateValue(0, 1, 8));
+  TestDateValue("next_day(date '0000-01-01', 'friday')", DateValue(0, 1, 7));
+  // 9999-12-31 is Friday
+  TestDateValue("next_day(date'9999-12-30', 'FRI')", DateValue(9999, 12, 31));
+  TestIsNull("next_day(date'9999-12-30', 'THU')", TYPE_DATE);
+  // Date is null
+  TestIsNull("next_day(cast(NULL as date), 'THU')", TYPE_DATE);
+  // Invalid day
+  for (const string day: { "", "S", "sa", "satu", "saturdayy" }) {
+    const string expr = "next_day(date '2019-06-05', '" + day + "')";
+    TestError(expr);
+  }
+  TestError("next_day(date '2019-06-05', NULL)");
+  TestError("next_day(cast(NULL as date), NULL)");
+
+  // last_day:
+  TestDateValue("last_day(date'2019-01-11')", DateValue(2019, 1, 31));
+  TestDateValue("last_day(date'2019-02-05')", DateValue(2019, 2, 28));
+  TestDateValue("last_day(date'2019-04-25')", DateValue(2019, 4, 30));
+  TestDateValue("last_day(date'2019-05-31')", DateValue(2019, 5, 31));
+  // 2016 is leap year
+  TestDateValue("last_day(date'2016-02-05')", DateValue(2016, 2, 29));
+  TestDateValue("last_day(date'0000-01-01')", DateValue(0, 1, 31));
+  TestDateValue("last_day(date'9999-12-31')", DateValue(9999, 12, 31));
+  TestIsNull("last_day(cast(NULL as date))", TYPE_DATE);
+
+  // years_add, years_sub:
+  TestDateValue("years_add(date '0125-05-24', 0)", DateValue(125, 5, 24));
+  TestDateValue("years_sub(date '0125-05-24', 0)", DateValue(125, 5, 24));
+  TestDateValue("years_add(date '0125-05-24', 125)", DateValue(250, 5, 24));
+  TestDateValue("years_add(date '0125-05-24', -125)", DateValue(0, 5, 24));
+  TestDateValue("years_sub(date '0125-05-24', 125)", DateValue(0, 5, 24));
+  // Test leap years.
+  TestDateValue("years_add(date '2000-02-29', 1)", DateValue(2001, 2, 28));
+  TestDateValue("years_add(date '2000-02-29', 4)", DateValue(2004, 2, 29));
+  TestDateValue("years_sub(date '2000-02-29', 1)", DateValue(1999, 2, 28));
+  TestDateValue("years_sub(date '2000-02-29', 4)", DateValue(1996, 2, 29));
+  // Test upper and lower limit
+  TestDateValue("years_add(date'0000-12-31', 9999)", DateValue(9999, 12, 31));
+  TestIsNull("years_add(date'0000-12-31', 10000)", TYPE_DATE);
+  TestDateValue("years_sub(date'9999-01-01', 9999)", DateValue(0, 1, 1));
+  TestIsNull("years_sub(date'9999-01-01', 10000)", TYPE_DATE);
+  // Test max int64
+  TestIsNull("years_add(date'0000-01-01', 2147483647)", TYPE_DATE);
+  TestIsNull("years_sub(date'9999-12-31', 2147483647)", TYPE_DATE);
+  // Test NULL values
+  TestIsNull("years_add(cast(NULL as date), 1)", TYPE_DATE);
+  TestIsNull("years_add(date '2019-01-01', NULL)", TYPE_DATE);
+  TestIsNull("years_add(cast(NULL as date), NULL)", TYPE_DATE);
+
+  // months_add, add_months, months_sub:
+  TestDateValue("months_add(date '0005-01-29', 0)", DateValue(5, 1, 29));
+  TestDateValue("months_sub(date '0005-01-29', 0)", DateValue(5, 1, 29));
+  TestDateValue("add_months(date '0005-01-29', -60)", DateValue(0, 1, 29));
+  TestDateValue("months_add(date '0005-01-29', -60)", DateValue(0, 1, 29));
+  TestDateValue("months_sub(date '0005-01-29', 60)", DateValue(0, 1, 29));
+  TestDateValue("add_months(date '9995-01-29', 59)", DateValue(9999, 12, 29));
+  TestDateValue("months_add(date '9995-01-29', 59)", DateValue(9999, 12, 29));
+  TestDateValue("months_sub(date '9995-01-29', -59)", DateValue(9999, 12, 29));
+  // If the input date falls on the last day of the month, the result will also always be
+  // the last day of the month.
+  TestDateValue("add_months(date '2000-02-29', 1)", DateValue(2000, 3, 31));
+  TestDateValue("add_months(date '1999-02-28', 12)", DateValue(2000, 2, 29));
+  TestDateValue("months_sub(date '2000-03-31', 1)", DateValue(2000, 2, 29));
+  TestDateValue("months_add(date '2000-03-31', -2)", DateValue(2000, 1, 31));
+  // Test upper and lower limit.
+  // 12 * 9999 == 119988
+  TestDateValue("months_add(date '0000-12-31', 119988)", DateValue(9999, 12, 31));
+  TestIsNull("months_add(date'0000-12-31', 119989)", TYPE_DATE);
+  TestDateValue("months_sub(date '9999-01-01', 119988)", DateValue(0, 1, 1));
+  TestIsNull("months_sub(date'9999-01-01', 119989)", TYPE_DATE);
+  // Test max int64
+  TestIsNull("months_add(date'0000-01-01', 2147483647)", TYPE_DATE);
+  TestIsNull("months_sub(date'9999-12-31', 2147483647)", TYPE_DATE);
+  // Test NULL values
+  TestIsNull("months_add(cast(NULL as date), 1)", TYPE_DATE);
+  TestIsNull("months_add(date '2019-01-01', NULL)", TYPE_DATE);
+  TestIsNull("months_add(cast(NULL as date), NULL)", TYPE_DATE);
+
+  // weeks_add, weeks_sub:
+  TestDateValue("weeks_add(date'2019-06-12', 0)", DateValue(2019, 6, 12));
+  TestDateValue("weeks_sub(date'2019-06-12', 0)", DateValue(2019, 6, 12));
+  TestDateValue("weeks_add(date'2019-06-12', 29)", DateValue(2020, 1, 1));
+  TestDateValue("weeks_add(date'2019-06-12', -24)", DateValue(2018, 12, 26));
+  TestDateValue("weeks_sub(date'2019-06-12', 24)", DateValue(2018, 12, 26));
+  // Test leap year
+  TestDateValue("weeks_add(date '2016-01-04', 8)", DateValue(2016, 2, 29));
+  // Test upper and ower limit. There are 3652424 days between 0000-01-01 and 9999-12-31.
+  // 3652424 days is 521774 weeks + 6 days.
+  TestDateValue("weeks_add(date'0000-01-01', 521774)", DateValue(9999, 12, 25));
+  TestIsNull("weeks_add(date'0000-01-01', 521775)", TYPE_DATE);
+  TestDateValue("weeks_sub(date'9999-12-31', 521774)", DateValue(0, 1, 7));
+  TestIsNull("weeks_sub(date'9999-12-31', 521775)", TYPE_DATE);
+  // Test max int64
+  TestIsNull("weeks_add(date'0000-01-01', 2147483647)", TYPE_DATE);
+  TestIsNull("weeks_sub(date'9999-12-31', 2147483647)", TYPE_DATE);
+  // Test NULL values
+  TestIsNull("weeks_sub(cast(NULL as date), 1)", TYPE_DATE);
+  TestIsNull("weeks_sub(date '2019-01-01', NULL)", TYPE_DATE);
+  TestIsNull("weeks_sub(cast(NULL as date), NULL)", TYPE_DATE);
+
+  // days_add, date_add, days_sub, date_sub, subdate:
+  TestDateValue("days_add(date'2019-06-12', 0)", DateValue(2019, 6, 12));
+  TestDateValue("days_sub(date'2019-06-12', 0)", DateValue(2019, 6, 12));
+  TestDateValue("date_add(date'2019-01-01', 365)", DateValue(2020, 1, 1));
+  TestDateValue("date_sub(date'2019-12-31', 365)", DateValue(2018, 12, 31));
+  // Test leap year
+  TestDateValue("date_add(date'2016-01-01', 366)", DateValue(2017, 1, 1));
+  TestDateValue("subdate(date'2016-12-31', 366)", DateValue(2015, 12, 31));
+  // Test uper and lower limit. There are 3652424 days between 0000-01-01 and 9999-12-31.
+  TestDateValue("days_add(date '0000-01-01', 3652424)", DateValue(9999, 12, 31));
+  TestIsNull("date_add(date '0000-01-01', 3652425)", TYPE_DATE);
+  TestDateValue("days_sub(date '9999-12-31', 3652424)", DateValue(0, 1, 1));
+  TestIsNull("date_sub(date '9999-12-31', 3652425)", TYPE_DATE);
+  // Test max int64
+  TestIsNull("days_add(date'0000-01-01', 2147483647)", TYPE_DATE);
+  TestIsNull("days_sub(date'9999-12-31', 2147483647)", TYPE_DATE);
+  // Test NULL values
+  TestIsNull("days_add(cast(NULL as date), 1)", TYPE_DATE);
+  TestIsNull("days_add(date '2019-01-01', NULL)", TYPE_DATE);
+  TestIsNull("days_add(cast(NULL as date), NULL)", TYPE_DATE);
+
+  // Interval expressions:
+  // Test year interval expressions.
+  TestDateValue("date_add(date '2000-02-29', interval 1 year)", DateValue(2001, 2, 28));
+  TestDateValue("date_add(date '2000-02-29', interval 4 year)", DateValue(2004, 2, 29));
+  TestDateValue("date_sub(date '2000-02-29', interval 1 year)", DateValue(1999, 2, 28));
+  TestDateValue("date_sub(date '2000-02-29', interval 4 year)", DateValue(1996, 2, 29));
+  TestDateValue("date '2000-02-29' + interval 1 year", DateValue(2001, 2, 28));
+  TestDateValue("date '2000-02-29' + interval 4 years", DateValue(2004, 2, 29));
+  TestDateValue("date '0000-12-31' + interval 9999 years", DateValue(9999, 12, 31));
+  TestIsNull("date '0000-12-31' + interval 10000 years", TYPE_DATE);
+  TestIsNull("date '0000-01-01' + interval 2147483647 years", TYPE_DATE);
+  // Test month interval expressions. Keep-last-day-of-month behavior is not enforced.
+  TestDateValue("date_add(date '2000-02-29', interval 1 month)", DateValue(2000, 3, 29));
+  TestDateValue("date_add(date '1999-02-28', interval 12 months)",
+      DateValue(2000, 2, 28));
+  TestDateValue("date_sub(date '2000-03-31', interval 1 month)", DateValue(2000, 2, 29));
+  TestDateValue("date_add(date '2000-03-31', interval -2 months)",
+      DateValue(2000, 1, 31));
+  TestDateValue("date '2000-02-29' + interval 1 month", DateValue(2000, 3, 29));
+  TestDateValue("date '2000-03-31' - interval 2 months", DateValue(2000, 1, 31));
+  TestDateValue("date '9999-01-01' - interval 119988 months", DateValue(0, 1, 1));
+  TestIsNull("date'9999-01-01' - interval 119989 months", TYPE_DATE);
+  TestIsNull("date'9999-12-31' - interval 2147483647 months", TYPE_DATE);
+  // Test week interval expressions.
+  TestDateValue("date_add(date'2019-06-12', interval -24 weeks)",
+      DateValue(2018, 12, 26));
+  TestDateValue("date_sub(date'2019-06-12', interval 24 weeks)", DateValue(2018, 12, 26));
+  TestDateValue("date_add(date '2016-01-04', interval 8 weeks)", DateValue(2016, 2, 29));
+  TestDateValue("date '2019-06-12' - interval 24 weeks", DateValue(2018, 12, 26));
+  TestDateValue("date '2018-12-26' + interval 24 weeks", DateValue(2019, 6, 12));
+  TestDateValue("date '9999-12-31' - interval 521774 weeks", DateValue(0, 1, 7));
+  TestIsNull("date '9999-12-31' - interval 521775 weeks", TYPE_DATE);
+  TestIsNull("date'9999-12-31' - interval 2147483647 weeks", TYPE_DATE);
+  // Test day interval expressions.
+  TestDateValue("date_add(date '2019-01-01', interval 365 days)", DateValue(2020, 1, 1));
+  TestDateValue("date_sub(date '2016-12-31', interval 366 days)",
+      DateValue(2015, 12, 31));
+  TestDateValue("date '0000-01-01' + interval 3652424 days", DateValue(9999, 12, 31));
+  TestIsNull("date '0000-01-01' + interval 3652425 days", TYPE_DATE);
+  TestIsNull("date '9999-12-31' - interval 2147483647 days", TYPE_DATE);
+  // Test NULL values.
+  TestIsNull("date_add(date '2019-01-01', interval cast(NULL as BIGINT) days)",
+      TYPE_DATE);
+  TestIsNull("date_add(cast(NULL as date), interval 1 days)", TYPE_DATE);
+  TestIsNull("date_add(cast(NULL as date), interval cast(NULL as BIGINT) days)",
+      TYPE_DATE);
+  TestIsNull("date '2019-01-01' - interval cast(NULL as BIGINT) days", TYPE_DATE);
+  TestIsNull("cast(NULL as date) - interval 1 days", TYPE_DATE);
+  TestIsNull("cast(NULL as date) - interval cast(NULL as BIGINT) days", TYPE_DATE);
+
+  // datediff:
+  TestValue("datediff(date'2019-05-12', date '2019-05-12')", TYPE_INT, 0);
+  TestValue("datediff(date'2020-01-01', '2019-01-01')", TYPE_INT, 365);
+  TestValue("datediff('2019-01-01', date '2020-01-01')", TYPE_INT, -365);
+  // Test leap year
+  TestValue("datediff(date'2021-01-01', date '2020-01-01')", TYPE_INT, 366);
+  TestValue("datediff('2020-01-01', date '2021-01-01')", TYPE_INT, -366);
+  // Test difference between min and max date
+  TestValue("datediff(date'9999-12-31', date '0000-01-01')", TYPE_INT, 3652424);
+  TestValue("datediff(date'0000-01-01', '9999-12-31')", TYPE_INT, -3652424);
+  // Test NULL values
+  TestIsNull("datediff(cast(NULL as DATE), date '0000-01-01')", TYPE_INT);
+  TestIsNull("datediff(date'9999-12-31', cast(NULL as date))", TYPE_INT);
+  TestIsNull("datediff(cast(NULL as DATE), cast(NULL as date))", TYPE_INT);
+
+  // date_cmp:
+  TestValue("date_cmp(date '2019-06-11', date '2019-06-11')", TYPE_INT, 0);
+  TestValue("date_cmp(date '2019-06-11', '2019-06-12')", TYPE_INT, -1);
+  TestValue("date_cmp('2019-06-12', date '2019-06-11')", TYPE_INT, 1);
+  // Test NULL values
+  TestIsNull("date_cmp(date '2019-06-12', cast(NULL as date))", TYPE_INT);
+  TestIsNull("date_cmp(cast(NULL as date), date '2019-06-11')", TYPE_INT);
+  TestIsNull("date_cmp(cast(NULL as DATE), cast(NULL as date))", TYPE_INT);
+  // Test upper and lower limit
+  TestValue("date_cmp(date '9999-12-31', '0000-01-01')", TYPE_INT, 1);
+
+  // int_months_between:
+  TestValue("int_months_between(date '1967-07-19','1966-06-04')", TYPE_INT, 13);
+  TestValue("int_months_between('1966-06-04', date'1967-07-19')", TYPE_INT, -13);
+  TestValue("int_months_between(date '1967-07-19','1967-07-19')", TYPE_INT, 0);
+  TestValue("int_months_between('2015-07-19', date '2015-08-18')", TYPE_INT, 0);
+  // Test lower and upper limit
+  TestValue("int_months_between(date '9999-12-31','0000-01-01')", TYPE_INT,
+      9999 * 12 + 11);
+  // Test NULL values
+  TestIsNull("int_months_between(date '1999-11-25', cast(NULL as date))", TYPE_INT);
+  TestIsNull("int_months_between(cast(NULL as DATE), date '1999-11-25')", TYPE_INT);
+  TestIsNull("int_months_between(cast(NULL as DATE), cast(NULL as date))", TYPE_INT);
+
+  // months_between:
+  TestValue("months_between(DATE '1967-07-19','1966-06-04')", TYPE_DOUBLE,
+      13.48387096774194);
+  TestValue("months_between('1966-06-04', date'1967-07-19')",
+      TYPE_DOUBLE, -13.48387096774194);
+  TestValue("months_between(date'1967-07-19','1967-07-19')", TYPE_DOUBLE, 0);
+  TestValue("months_between(date'2015-02-28','2015-05-31')", TYPE_DOUBLE, -3);
+  TestValue("months_between(date'2012-02-29','2012-01-31')", TYPE_DOUBLE, 1);
+  // Test NULL values
+  TestIsNull("months_between(date '1999-11-25', cast(NULL as date))", TYPE_DOUBLE);
+  TestIsNull("months_between(cast(NULL as DATE), date '1999-11-25')", TYPE_DOUBLE);
+  TestIsNull("months_between(cast(NULL as DATE), cast(NULL as date))", TYPE_DOUBLE);
+
+  // current_date:
+  // Test that current_date() is reasonable.
+  {
+    ScopedTimeZoneOverride time_zone(TEST_TZ_WITHOUT_DST);
+    ScopedLocalUnixTimestampConversionOverride use_local;
+    const Timezone& local_tz = time_zone.GetTimezone();
+
+    const boost::gregorian::date start_date =
+        TimestampValue::FromUnixTimeMicros(UnixMicros(), local_tz).date();
+    DateValue current_dv = ConvertValue<DateValue>(GetValue("current_date()", TYPE_DATE));
+    const boost::gregorian::date end_date =
+        TimestampValue::FromUnixTimeMicros(UnixMicros(), local_tz).date();
+
+    int year, month, day;
+    EXPECT_TRUE(current_dv.ToYearMonthDay(&year, &month, &day));
+    const boost::gregorian::date current_date(year, month, day);
+    EXPECT_BETWEEN(start_date, current_date, end_date);
+  }
 }
 
 TEST_P(ExprTest, ConditionalFunctions) {
diff --git a/be/src/exprs/scalar-expr-evaluator.cc b/be/src/exprs/scalar-expr-evaluator.cc
index 40c971d..40723e7 100644
--- a/be/src/exprs/scalar-expr-evaluator.cc
+++ b/be/src/exprs/scalar-expr-evaluator.cc
@@ -28,6 +28,7 @@
 #include "exprs/cast-functions.h"
 #include "exprs/compound-predicates.h"
 #include "exprs/conditional-functions.h"
+#include "exprs/date-functions.h"
 #include "exprs/decimal-functions.h"
 #include "exprs/decimal-operators.h"
 #include "exprs/hive-udf-call.h"
@@ -439,6 +440,7 @@ void ScalarExprEvaluator::InitBuiltinsDummy() {
   StringFunctions::Length(nullptr, StringVal::null());
   TimestampFunctions::Year(nullptr, TimestampVal::null());
   TimestampFunctions::UnixAndFromUnixPrepare(nullptr, FunctionContext::FRAGMENT_LOCAL);
+  DateFunctions::Year(nullptr, DateVal::null());
   UdfBuiltins::Pi(nullptr);
   UtilityFunctions::Pid(nullptr);
 }
diff --git a/be/src/exprs/timestamp-functions-ir.cc b/be/src/exprs/timestamp-functions-ir.cc
index 5b1dcfa..8426b0e 100644
--- a/be/src/exprs/timestamp-functions-ir.cc
+++ b/be/src/exprs/timestamp-functions-ir.cc
@@ -580,28 +580,23 @@ DoubleVal TimestampFunctions::MonthsBetween(FunctionContext* context,
 
 TimestampVal TimestampFunctions::NextDay(FunctionContext* context,
     const TimestampVal& date, const StringVal& weekday) {
+  if (weekday.is_null) {
+    context->SetError("Invalid Day: NULL");
+    return TimestampVal::null();
+  }
+
   string weekday_str = string(reinterpret_cast<const char*>(weekday.ptr), weekday.len);
   transform(weekday_str.begin(), weekday_str.end(), weekday_str.begin(), tolower);
-  int day_idx = 0;
-  if (weekday_str == "sunday" || weekday_str == "sun") {
-    day_idx = 1;
-  } else if (weekday_str == "monday" || weekday_str == "mon") {
-    day_idx = 2;
-  } else if (weekday_str == "tuesday" || weekday_str == "tue") {
-    day_idx = 3;
-  } else if (weekday_str == "wednesday" || weekday_str == "wed") {
-    day_idx = 4;
-  } else if (weekday_str == "thursday" || weekday_str == "thu") {
-    day_idx = 5;
-  } else if (weekday_str == "friday" || weekday_str == "fri") {
-    day_idx = 6;
-  } else if (weekday_str == "saturday" || weekday_str == "sat") {
-    day_idx = 7;
+
+  const auto it = DAYNAME_MAP.find(weekday_str);
+  if (it == DAYNAME_MAP.end()) {
+    context->SetError(Substitute("Invalid Day: $0", weekday_str).c_str());
+    return TimestampVal::null();
   }
-  DCHECK_GE(day_idx, 1);
-  DCHECK_LE(day_idx, 7);
+  DCHECK_GE(it->second, 0);
+  DCHECK_LE(it->second, 6);
 
-  int delta_days = day_idx - DayOfWeek(context, date).val;
+  int delta_days = it->second + 1 - DayOfWeek(context, date).val;
   delta_days = delta_days <= 0 ? delta_days + 7 : delta_days;
   DCHECK_GE(delta_days, 1);
   DCHECK_LE(delta_days, 7);
diff --git a/be/src/exprs/timestamp-functions.cc b/be/src/exprs/timestamp-functions.cc
index 121b69d..97bf1df 100644
--- a/be/src/exprs/timestamp-functions.cc
+++ b/be/src/exprs/timestamp-functions.cc
@@ -45,6 +45,17 @@ const string TimestampFunctions::MONTHNAME_ARRAY[12] = {"January", "February", "
     "April", "May", "June", "July", "August", "September", "October", "November",
     "December"};
 
+// Sunday is mapped to 0 and Saturday is mapped to 6.
+const map<string, int> TimestampFunctions::DAYNAME_MAP = {
+    {"sun", 0}, {"sunday", 0},
+    {"mon", 1}, {"monday", 1},
+    {"tue", 2}, {"tuesday", 2},
+    {"wed", 3}, {"wednesday", 3},
+    {"thu", 4}, {"thursday", 4},
+    {"fri", 5}, {"friday", 5},
+    {"sat", 6}, {"saturday", 6},
+};
+
 TimestampVal TimestampFunctions::FromUtc(FunctionContext* context,
     const TimestampVal& ts_val, const StringVal& tz_string_val) {
   if (ts_val.is_null || tz_string_val.is_null) return TimestampVal::null();
diff --git a/be/src/exprs/timestamp-functions.h b/be/src/exprs/timestamp-functions.h
index 9c63f00..3beadf1 100644
--- a/be/src/exprs/timestamp-functions.h
+++ b/be/src/exprs/timestamp-functions.h
@@ -19,6 +19,8 @@
 #ifndef IMPALA_EXPRS_TIMESTAMP_FUNCTIONS_H
 #define IMPALA_EXPRS_TIMESTAMP_FUNCTIONS_H
 
+#include <map>
+
 #include "common/status.h"
 #include "udf/udf.h"
 
@@ -68,6 +70,15 @@ class TimestampFunctions {
   /// Static result values for ShortMonthName() function.
   /// Short month names are also used in DateParser.
   static const std::string MONTH_ARRAY[12];
+  /// Static result values for ShortDayName() and LongMonthName() functions.
+  /// These are also used in DateFunctions.
+  static const std::string DAYNAME_ARRAY[7];
+  static const std::string MONTHNAME_ARRAY[12];
+
+  /// Maps full and abbreviated lowercase names of day-of-week to an int in the 0-6 range.
+  /// Sunday is mapped to 0 and Saturday is mapped to 6.
+  /// It is used in NextDay() function.
+  static const std::map<std::string, int> DAYNAME_MAP;
 
   /// Parse and initialize format string if it is a constant. Raise error if invalid.
   static void UnixAndFromUnixPrepare(FunctionContext* context,
@@ -233,10 +244,8 @@ class TimestampFunctions {
       const StringVal& format, bool is_error);
 
  private:
-  /// Static result values for DayName(), ShortDayName() and LongMonthName() functions.
+  /// Static result values for DayName() function.
   static const std::string DAY_ARRAY[7];
-  static const std::string DAYNAME_ARRAY[7];
-  static const std::string MONTHNAME_ARRAY[12];
 };
 
 } // namespace impala
diff --git a/be/src/exprs/udf-builtins.cc b/be/src/exprs/udf-builtins.cc
index 786e9cf..362f16b 100644
--- a/be/src/exprs/udf-builtins.cc
+++ b/be/src/exprs/udf-builtins.cc
@@ -677,6 +677,9 @@ ReturnUdfType ExtractTruncFuncTempl(FunctionContext* ctx, const UdfType& val,
   void* state = ctx->GetFunctionState(FunctionContext::THREAD_LOCAL);
   if (state != NULL) {
     unit = *reinterpret_cast<UnitType*>(state);
+  } else if (unit_str.is_null) {
+    ctx->SetError(Substitute("Invalid $0 $1: NULL", func_descr, unit_descr).c_str());
+    return ReturnUdfType::null();
   } else {
     unit = to_unit(ctx, unit_str);
     if (!ALLOW_TIME_OF_DAY_UNIT && IsTimeOfDayUnit(unit)) {
@@ -712,20 +715,24 @@ void ExtractTruncFuncPrepareTempl(FunctionContext* ctx,
   // Parse the unit up front if we can, otherwise do it on the fly in trunc_templ()
   if (ctx->IsArgConstant(UNIT_IDX)) {
     StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(UNIT_IDX));
-    UnitType unit = to_unit(ctx, *unit_str);
-    if (!ALLOW_TIME_OF_DAY_UNIT && IsTimeOfDayUnit(unit)) {
-      string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute(
-          "Unsupported $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
-    } else if (IsInvalidUnit(unit)) {
-      string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute(
-          "Invalid $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+    if (unit_str == nullptr || unit_str->is_null) {
+      ctx->SetError(Substitute("Invalid $0 $1: NULL", func_descr, unit_descr).c_str());
     } else {
-      UnitType* state = ctx->Allocate<UnitType>();
-      RETURN_IF_NULL(ctx, state);
-      *state = unit;
-      ctx->SetFunctionState(scope, state);
+      UnitType unit = to_unit(ctx, *unit_str);
+      if (!ALLOW_TIME_OF_DAY_UNIT && IsTimeOfDayUnit(unit)) {
+        string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
+        ctx->SetError(Substitute(
+            "Unsupported $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+      } else if (IsInvalidUnit(unit)) {
+        string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
+        ctx->SetError(Substitute(
+            "Invalid $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+      } else {
+        UnitType* state = ctx->Allocate<UnitType>();
+        RETURN_IF_NULL(ctx, state);
+        *state = unit;
+        ctx->SetFunctionState(scope, state);
+      }
     }
   }
 }
diff --git a/be/src/runtime/date-test.cc b/be/src/runtime/date-test.cc
index 0b8f804..13b5c68 100644
--- a/be/src/runtime/date-test.cc
+++ b/be/src/runtime/date-test.cc
@@ -18,6 +18,7 @@
 #include <boost/date_time/gregorian/gregorian.hpp>
 #include <boost/date_time/posix_time/posix_time.hpp>
 
+#include "cctz/civil_time.h"
 #include "common/status.h"
 #include "runtime/date-value.h"
 #include "runtime/datetime-parse-util.h"
@@ -610,74 +611,355 @@ TEST(DateTest, AddDays) {
   EXPECT_EQ(DateValue(2020, 5, 16), dv.AddDays(366));
   EXPECT_EQ(DateValue(2018, 5, 16), dv.AddDays(-365));
 
-  // Test upper limit
+  // Test upper limit.
   dv = DateValue(9999, 12, 20);
   EXPECT_EQ(DateValue(9999, 12, 31), dv.AddDays(11));
   EXPECT_FALSE(dv.AddDays(12).IsValid());
   EXPECT_FALSE(dv.AddDays(13).IsValid());
 
-  // Test lower limit
+  // Test lower limit.
   dv = DateValue(0, 1, 10);
   EXPECT_EQ(DateValue(0, 1, 1), dv.AddDays(-9));
   EXPECT_FALSE(dv.AddDays(-10).IsValid());
   EXPECT_FALSE(dv.AddDays(-11).IsValid());
 
-  // Test leap year
+  // Test adding days to cover the entire range.
+  int32_t min_dse, max_dse;
+  EXPECT_TRUE(DateValue(0, 1, 1).ToDaysSinceEpoch(&min_dse));
+  EXPECT_GT(0, min_dse);
+  min_dse = -min_dse;
+  EXPECT_TRUE(DateValue(9999, 12, 31).ToDaysSinceEpoch(&max_dse));
+  EXPECT_LT(0, max_dse);
+
+  dv = DateValue(0, 1, 1);
+  EXPECT_EQ(DateValue(9999, 12, 31), dv.AddDays(min_dse + max_dse));
+  EXPECT_FALSE(dv.AddDays(min_dse + max_dse + 1).IsValid());
+  EXPECT_FALSE(dv.AddDays(std::numeric_limits<int64_t>::max()).IsValid());
+
+  dv = DateValue(9999, 12, 31);
+  EXPECT_EQ(DateValue(0, 1, 1), dv.AddDays(-(min_dse + max_dse)));
+  EXPECT_FALSE(dv.AddDays(-(min_dse + max_dse + 1)).IsValid());
+  EXPECT_FALSE(dv.AddDays(std::numeric_limits<int64_t>::min()).IsValid());
+
+  // Test leap year.
   dv = DateValue(2000, 2, 20);
   EXPECT_EQ(DateValue(2000, 2, 28), dv.AddDays(8));
   EXPECT_EQ(DateValue(2000, 2, 29), dv.AddDays(9));
   EXPECT_EQ(DateValue(2000, 3, 1), dv.AddDays(10));
 
-  // Test non-leap year
+  // Test non-leap year.
   dv = DateValue(2001, 2, 20);
   EXPECT_EQ(DateValue(2001, 2, 28), dv.AddDays(8));
   EXPECT_EQ(DateValue(2001, 3, 1), dv.AddDays(9));
 }
 
+TEST(DateTest, AddMonths) {
+  // Adding days to an invalid DateValue instance returns an invalid DateValue.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.IsValid());
+  EXPECT_FALSE(invalid_dv.AddMonths(1, true).IsValid());
+
+  // AddMonths works with 0, > 0 and < 0 number of months.
+  DateValue dv(2019, 5, 16);
+  EXPECT_EQ(DateValue(2019, 6, 16), dv.AddMonths(1, true));
+  EXPECT_EQ(DateValue(2019, 4, 16), dv.AddMonths(-1, true));
+
+  // Test that result dates are always capped at the end of the month regardless of
+  // whether 'keep_last_day' is set or not.
+  dv = DateValue(2019, 5, 31);
+  EXPECT_EQ(DateValue(2019, 6, 30), dv.AddMonths(1, true));
+  EXPECT_EQ(DateValue(2019, 7, 31), dv.AddMonths(2, true));
+  EXPECT_EQ(DateValue(2020, 2, 29), dv.AddMonths(9, true));
+  EXPECT_EQ(DateValue(2019, 6, 30), dv.AddMonths(1, false));
+  EXPECT_EQ(DateValue(2019, 7, 31), dv.AddMonths(2, false));
+  EXPECT_EQ(DateValue(2020, 2, 29), dv.AddMonths(9, false));
+
+  // Test that resulting date falls on the last day iff 'keep_last_day' is set.
+  dv = DateValue(1999, 2, 28);
+  EXPECT_EQ(DateValue(1999, 3, 31), dv.AddMonths(1, true));
+  EXPECT_EQ(DateValue(1999, 4, 30), dv.AddMonths(2, true));
+  EXPECT_EQ(DateValue(2000, 2, 29), dv.AddMonths(12, true));
+  EXPECT_EQ(DateValue(1999, 3, 28), dv.AddMonths(1, false));
+  EXPECT_EQ(DateValue(1999, 4, 28), dv.AddMonths(2, false));
+  EXPECT_EQ(DateValue(2000, 2, 28), dv.AddMonths(12, false));
+
+  // Test that leap year is handled correctly.
+  dv = DateValue(2016, 2, 29);
+  EXPECT_EQ(DateValue(2016, 3, 31), dv.AddMonths(1, true));
+  EXPECT_EQ(DateValue(2016, 4, 30), dv.AddMonths(2, true));
+  EXPECT_EQ(DateValue(2017, 2, 28), dv.AddMonths(12, true));
+  EXPECT_EQ(DateValue(2016, 3, 29), dv.AddMonths(1, false));
+  EXPECT_EQ(DateValue(2016, 4, 29), dv.AddMonths(2, false));
+  EXPECT_EQ(DateValue(2017, 2, 28), dv.AddMonths(12, false));
+
+  // Test upper limit.
+  dv = DateValue(9998, 11, 30);
+  EXPECT_EQ(DateValue(9999, 12, 31), dv.AddMonths(13, true));
+  EXPECT_FALSE(dv.AddMonths(14, true).IsValid());
+
+  // Test lower limit.
+  dv = DateValue(0, 11, 30);
+  EXPECT_EQ(DateValue(0, 1, 31), dv.AddMonths(-10, true));
+  EXPECT_FALSE(dv.AddMonths(-11, true).IsValid());
+
+  // Test adding months to cover the entire range.
+  dv = DateValue(0, 1, 1);
+  EXPECT_EQ(DateValue(9999, 12, 1), dv.AddMonths(9999 * 12 + 11, false));
+  EXPECT_FALSE(dv.AddMonths(9999 * 12 + 12, false).IsValid());
+  EXPECT_FALSE(dv.AddMonths(std::numeric_limits<int64_t>::max(), false).IsValid());
+
+  dv = DateValue(9999, 12, 31);
+  EXPECT_EQ(DateValue(0, 1, 31), dv.AddMonths(-9999 * 12 - 11, false));
+  EXPECT_FALSE(dv.AddMonths(-9999 * 12 - 12, false).IsValid());
+  EXPECT_FALSE(dv.AddMonths(std::numeric_limits<int64_t>::min(), false).IsValid());
+}
+
+TEST(DateTest, AddYears) {
+  // Adding years to an invalid DateValue instance returns an invalid DateValue.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.IsValid());
+  EXPECT_FALSE(invalid_dv.AddYears(1).IsValid());
+
+  // AddYears works with 0, > 0 and < 0 number of days.
+  DateValue dv(2019, 5, 16);
+  EXPECT_EQ(DateValue(2020, 5, 16), dv.AddYears(1));
+  EXPECT_EQ(DateValue(2018, 5, 16), dv.AddYears(-1));
+
+  // Test upper limit.
+  dv = DateValue(9990, 12, 31);
+  EXPECT_EQ(DateValue(9999, 12, 31), dv.AddYears(9));
+  EXPECT_FALSE(dv.AddYears(10).IsValid());
+  EXPECT_FALSE(dv.AddYears(11).IsValid());
+
+  // Test lower limit.
+  dv = DateValue(10, 1, 1);
+  EXPECT_EQ(DateValue(0, 1, 1), dv.AddYears(-10));
+  EXPECT_FALSE(dv.AddYears(-11).IsValid());
+  EXPECT_FALSE(dv.AddYears(-12).IsValid());
+
+  // Test adding years to cover the entire range.
+  dv = DateValue(0, 1, 1);
+  EXPECT_EQ(DateValue(9999, 1, 1), dv.AddYears(9999));
+  EXPECT_FALSE(dv.AddYears(9999 + 1).IsValid());
+  EXPECT_FALSE(dv.AddYears(std::numeric_limits<int64_t>::max()).IsValid());
+
+  dv = DateValue(9999, 12, 31);
+  EXPECT_EQ(DateValue(0, 12, 31), dv.AddYears(-9999));
+  EXPECT_FALSE(dv.AddYears(-9999 - 1).IsValid());
+  EXPECT_FALSE(dv.AddYears(std::numeric_limits<int64_t>::min()).IsValid());
+
+  // Test leap year.
+  dv = DateValue(2000, 2, 29);
+  EXPECT_EQ(DateValue(2001, 2, 28), dv.AddYears(1));
+  EXPECT_EQ(DateValue(2002, 2, 28), dv.AddYears(2));
+  EXPECT_EQ(DateValue(2003, 2, 28), dv.AddYears(3));
+  EXPECT_EQ(DateValue(2004, 2, 29), dv.AddYears(4));
+}
+
 TEST(DateTest, WeekDay) {
   // WeekDay() returns -1 for invalid dates.
   DateValue invalid_dv;
   EXPECT_FALSE(invalid_dv.IsValid());
   EXPECT_EQ(-1, invalid_dv.WeekDay());
 
-  // 2019.05.01 is Wednesday.
+  // 2019-05-01 is Wednesday.
   DateValue dv(2019, 5, 1);
   for (int i = 0; i <= 31; ++i) {
     // 0 = Monday, 2 = Wednesday and 6 = Sunday.
     EXPECT_EQ((i + 2) % 7, dv.AddDays(i).WeekDay());
   }
 
-  // Test upper limit. 9999.12.31 is Friday.
+  // Test upper limit. 9999-12-31 is Friday.
   EXPECT_EQ(4, DateValue(9999, 12, 31).WeekDay());
 
   // Test lower limit.
-  // 0000.01.01 is Monday.
+  // 0001-01-01 is Monday.
   EXPECT_EQ(0, DateValue(1, 1, 1).WeekDay());
-  // 0000.01.01 is Saturday.
+  // 0000-01-01 is Saturday.
   EXPECT_EQ(5, DateValue(0, 1, 1).WeekDay());
 }
 
-TEST(DateTest, ToYear) {
-  int year;
-
-  // Test that ToYear() returns false for invalid dates.
+TEST(DateTest, ToYearMonthDay) {
+  // Test that ToYearMonthDay() and ToYear() return false for invalid dates.
   DateValue invalid_dv;
   EXPECT_FALSE(invalid_dv.IsValid());
-  EXPECT_FALSE(invalid_dv.ToYear(&year));
-
-  // Test that ToYear() returns the same year as ToYearMonthDay().
-  // The following loop iterates through all valid dates:
-  DateValue dv(0, 1, 1);
-  EXPECT_TRUE(dv.IsValid());
+  int y1, m1, d1;
+  EXPECT_FALSE(invalid_dv.ToYearMonthDay(&y1, &m1, &d1));
+  int y2;
+  EXPECT_FALSE(invalid_dv.ToYear(&y2));
+
+  // Test that ToYearMonthDay() and ToYear() return the same values as
+  // cctz::civil_day::year()/month()/day().
+  // The following loop iterates through all valid dates (0000-01-01..9999-12-31):
+  cctz::civil_day epoch(1970, 1, 1);
+  cctz::civil_day cd(0, 1, 1);
   do {
-    int y, m, d;
-    EXPECT_TRUE(dv.ToYearMonthDay(&y, &m, &d));
+    DateValue dv(cd - epoch);
+    EXPECT_TRUE(dv.IsValid());
+
+    EXPECT_TRUE(dv.ToYearMonthDay(&y1, &m1, &d1));
+    EXPECT_EQ(cd.year(), y1);
+    EXPECT_EQ(cd.month(), m1);
+    EXPECT_EQ(cd.day(), d1);
+
+    EXPECT_TRUE(dv.ToYear(&y2));
+    EXPECT_EQ(cd.year(), y2);
+
+    cd++;
+  } while (cd.year() < 10000);
+}
 
-    EXPECT_TRUE(dv.ToYear(&year));
-    EXPECT_EQ(y, year);
+TEST(DateTest, DayOfYear) {
+  DateValue invalid_dv;
+  EXPECT_EQ(-1, invalid_dv.DayOfYear());
+
+  // Test lower limit.
+  EXPECT_EQ(1, DateValue(0, 1, 1).DayOfYear());
+  // Test upper limit.
+  EXPECT_EQ(365, DateValue(9999, 12,31).DayOfYear());
+
+  // Test leap year.
+  EXPECT_EQ(1, DateValue(2000, 1, 1).DayOfYear());
+  EXPECT_EQ(31, DateValue(2000, 1, 31).DayOfYear());
+  EXPECT_EQ(32, DateValue(2000, 2, 1).DayOfYear());
+  EXPECT_EQ(59, DateValue(2000, 2, 28).DayOfYear());
+  EXPECT_EQ(60, DateValue(2000, 2, 29).DayOfYear());
+  EXPECT_EQ(61, DateValue(2000, 3, 1).DayOfYear());
+  EXPECT_EQ(366, DateValue(2000, 12, 31).DayOfYear());
+}
+
+TEST(DateTest, WeekOfYear) {
+  // Test that it returns -1 for invalid dates.
+  DateValue invalid_dv;
+  EXPECT_EQ(-1, invalid_dv.WeekOfYear());
+
+  // Iterate through days of 2019.
+  // 2019-01-01 is Tuesday and 2019-12-31 is Tuesday too.
+  DateValue jan1(2019, 1, 1);
+  int weekday_offset = 1;
+  for (DateValue dv = jan1;
+      dv <= DateValue(2019, 12, 29);
+      dv = dv.AddDays(1)) {
+    EXPECT_EQ(weekday_offset / 7 + 1, dv.WeekOfYear());
+    ++weekday_offset;
+  }
 
-    dv = dv.AddDays(1);
-  } while (dv.IsValid());
+  // Year 2015 has 53 weeks. 2015-12-31 is Thursday.
+  EXPECT_EQ(53, DateValue(2015, 12, 31).WeekOfYear());
+
+  // 2019-12-30 (Monday) and 2019-12-31 (Tuesday) belong to year 2020.
+  EXPECT_EQ(1, DateValue(2019, 12, 30).WeekOfYear());
+  EXPECT_EQ(1, DateValue(2019, 12, 31).WeekOfYear());
+  EXPECT_EQ(1, DateValue(2020, 1, 1).WeekOfYear());
+  EXPECT_EQ(1, DateValue(2020, 1, 5).WeekOfYear());
+  EXPECT_EQ(2, DateValue(2020, 1, 6).WeekOfYear());
+
+  // 0001-01-01 is Monday. Test days around 0001-01-01.
+  EXPECT_EQ(51, DateValue(0, 12, 24).WeekOfYear());
+  EXPECT_EQ(52, DateValue(0, 12, 25).WeekOfYear());
+  EXPECT_EQ(52, DateValue(0, 12, 31).WeekOfYear());
+  EXPECT_EQ(1, DateValue(1, 1, 1).WeekOfYear());
+  EXPECT_EQ(1, DateValue(1, 1, 7).WeekOfYear());
+  EXPECT_EQ(2, DateValue(1, 1, 8).WeekOfYear());
+  // 0000-01-01 is Saturday. Test days around 0000-01-01.
+  EXPECT_EQ(52, DateValue(0, 1, 1).WeekOfYear());
+  EXPECT_EQ(52, DateValue(0, 1, 2).WeekOfYear());
+  EXPECT_EQ(1, DateValue(0, 1, 3).WeekOfYear());
+
+  // 9999-12-31 is Friday. Test days around 9999-12-31.
+  EXPECT_EQ(52, DateValue(9999, 12, 31).WeekOfYear());
+  EXPECT_EQ(52, DateValue(9999, 12, 27).WeekOfYear());
+  EXPECT_EQ(51, DateValue(9999, 12, 26).WeekOfYear());
+}
+
+TEST(DateTest, LastDay) {
+  // Test that it returns invalid DateValue for invalid dates.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.LastDay().IsValid());
+
+  // Test a non-leap year.
+  int month_days[] = {31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31};
+  for (DateValue dv(2019, 1, 1); dv <= DateValue(2019, 12, 31); dv = dv.AddDays(1)) {
+    int year, month, day;
+    EXPECT_TRUE(dv.ToYearMonthDay(&year, &month, &day));
+    EXPECT_EQ(DateValue(year, month, month_days[month - 1]), dv.LastDay());
+  }
+
+  // Test a leap year.
+  month_days[1] = 29;
+  for (DateValue dv(2016, 1, 1); dv <= DateValue(2016, 12, 31); dv = dv.AddDays(1)) {
+    int year, month, day;
+    EXPECT_TRUE(dv.ToYearMonthDay(&year, &month, &day));
+    EXPECT_EQ(DateValue(year, month, month_days[month - 1]), dv.LastDay());
+  }
+
+  // Test upper limit.
+  EXPECT_EQ(DateValue(9999, 12, 31), DateValue(9999, 12, 1).LastDay());
+  EXPECT_EQ(DateValue(9999, 12, 31), DateValue(9999, 12, 31).LastDay());
+
+  // Test lower limit.
+  EXPECT_EQ(DateValue(0, 1, 31), DateValue(0, 1, 1).LastDay());
+  EXPECT_EQ(DateValue(0, 1, 31), DateValue(0, 1, 31).LastDay());
+}
+
+// These macros add scoped trace to provide the line number of the caller upon failure.
+#define TEST_MONTHS_BW_RANGE(date1, date2, min_expected, max_expected) { \
+    SCOPED_TRACE(""); \
+    TestMonthsBetween((date1), (date2), (min_expected), (max_expected)); \
+  }
+
+#define TEST_MONTHS_BW(date1, date2, expected) { \
+    SCOPED_TRACE(""); \
+    TestMonthsBetween((date1), (date2), (expected), (expected)); \
+  }
+
+void TestMonthsBetween(const DateValue& dv1, const DateValue& dv2, double min_expected,
+    double max_expected) {
+  double months_between;
+  EXPECT_TRUE(dv1.MonthsBetween(dv2, &months_between));
+  EXPECT_LE(min_expected, months_between);
+  EXPECT_LE(months_between, max_expected);
+}
+
+TEST(DateTest, MonthsBetween) {
+  DateValue invalid_dv;
+  double months_between;
+  EXPECT_FALSE(invalid_dv.MonthsBetween(DateValue(), &months_between));
+  EXPECT_FALSE(invalid_dv.MonthsBetween(DateValue(2001, 1, 1), &months_between));
+  EXPECT_FALSE(DateValue(2001, 1, 1).MonthsBetween(invalid_dv, &months_between));
+
+  // Test that if both dates are on the same day of the month, the result has no
+  // fractional part.
+  TEST_MONTHS_BW(DateValue(2016, 2, 29), DateValue(2016, 1, 29), 1);
+  TEST_MONTHS_BW(DateValue(2016, 2, 29), DateValue(2016, 3, 29), -1);
+
+  // Test that if both dates are on the last day of the month, the result has no
+  // fractional part.
+  TEST_MONTHS_BW(DateValue(2016, 2, 29), DateValue(2016, 1, 31), 1);
+  TEST_MONTHS_BW(DateValue(2016, 2, 29), DateValue(2016, 3, 31), -1);
+
+  // Otherwise, there's a fractional part.
+  // There are 30/31.0 months between 2016-02-29 and 2016-01-30.
+  TEST_MONTHS_BW_RANGE(DateValue(2016, 2, 29), DateValue(2016, 1, 30), 29/31.0, 1.0);
+  // There are -32/31.0 months between 2016-02-29 and 2016-03-30.
+  TEST_MONTHS_BW_RANGE(DateValue(2016, 2, 29), DateValue(2016, 3, 30), -33/31.0, -1.0);
+  // There are 28/31.0 months between 2016-02-29 and 2016-02-01.
+  TEST_MONTHS_BW_RANGE(DateValue(2016, 2, 29), DateValue(2016, 2, 1), 27/31.0, 29/31.0);
+  // There are -30/31.0 months between 2016-02-29 and 2016-03-28.
+  TEST_MONTHS_BW_RANGE(DateValue(2016, 2, 29), DateValue(2016, 3, 28), -31/31.0,
+      -29/31.0);
+
+  // Test entire range w/o fractional part.
+  TEST_MONTHS_BW(DateValue(0, 1, 1), DateValue(9999, 12, 1), -9999 * 12 - 11);
+  TEST_MONTHS_BW(DateValue(9999, 12, 31), DateValue(0, 1, 31), 9999 * 12 + 11);
+
+  // Test entire range w/ fractional part.
+  // There are (-9999*12 - 11 - 30/31.0) months between 0000-01-01 and 9999-12-31.
+  TEST_MONTHS_BW_RANGE(DateValue(0, 1, 1), DateValue(9999, 12, 31), -10000 * 12.0,
+      -9999 * 12 - 11 - 29/31.0);
+  // There are (9999*12 + 11 + 30/31.0) months between 9999-12-31 and 0000-01-01.
+  TEST_MONTHS_BW_RANGE(DateValue(9999, 12, 31), DateValue(0, 1, 1),
+      9999 * 12 + 11 + 29/31.0, 10000 * 12.0);
 }
 
 }
diff --git a/be/src/runtime/date-value.cc b/be/src/runtime/date-value.cc
index 54b31da..13cf2ce 100644
--- a/be/src/runtime/date-value.cc
+++ b/be/src/runtime/date-value.cc
@@ -19,7 +19,6 @@
 #include "runtime/date-value.h"
 
 #include <iomanip>
-
 #include "cctz/civil_time.h"
 #include "runtime/date-parse-util.h"
 
@@ -27,30 +26,31 @@
 
 namespace impala {
 
-namespace {
-const int EPOCH_YEAR = 1970;
-const cctz::civil_day EPOCH_DATE(EPOCH_YEAR, 1, 1);
-
-inline int32_t CalcDaysSinceEpoch(const cctz::civil_day& date) {
-  return date - EPOCH_DATE;
-}
-
-}
-
 using datetime_parse_util::DateTimeFormatContext;
 
-const int DateValue::MIN_YEAR = 0;
-const int DateValue::MAX_YEAR = 9999;
+const int EPOCH_YEAR = 1970;
+const int MIN_YEAR = 0;
+const int MAX_YEAR = 9999;
+
+const cctz::civil_day EPOCH_DATE(EPOCH_YEAR, 1, 1);
 
-const int32_t DateValue::MIN_DAYS_SINCE_EPOCH = CalcDaysSinceEpoch(
-    cctz::civil_day(MIN_YEAR, 1, 1));
-const int32_t DateValue::MAX_DAYS_SINCE_EPOCH = CalcDaysSinceEpoch(
-    cctz::civil_day(MAX_YEAR, 12, 31));
+const int32_t DateValue::MIN_DAYS_SINCE_EPOCH =
+    cctz::civil_day(MIN_YEAR, 1, 1) - EPOCH_DATE;
+const int32_t DateValue::MAX_DAYS_SINCE_EPOCH =
+    cctz::civil_day(MAX_YEAR, 12, 31) - EPOCH_DATE;
 
 const DateValue DateValue::MIN_DATE(MIN_DAYS_SINCE_EPOCH);
 const DateValue DateValue::MAX_DATE(MAX_DAYS_SINCE_EPOCH);
 
-DateValue::DateValue(int year, int month, int day)
+// Describes ranges for months in a non-leap year expressed as number of days since
+// January 1.
+const vector<int> MONTH_RANGES = {
+    0, 31, 59, 90, 120, 151, 181, 212, 243, 273, 304, 334, 365 };
+// Describes ranges for months in a leap year expressed as number of days since January 1.
+const vector<int> LEAP_YEAR_MONTH_RANGES = {
+    0, 31, 60, 91, 121, 152, 182, 213, 244, 274, 305, 335, 366 };
+
+DateValue::DateValue(int64_t year, int64_t month, int64_t day)
     : days_since_epoch_(INVALID_DAYS_SINCE_EPOCH) {
   DCHECK(!IsValid());
   // Check year range and whether year-month-day is a valid date.
@@ -58,7 +58,7 @@ DateValue::DateValue(int year, int month, int day)
     // Use CCTZ for validity check.
     cctz::civil_day date(year, month, day);
     if (LIKELY(year == date.year() && month == date.month() && day == date.day())) {
-      days_since_epoch_ = CalcDaysSinceEpoch(date);
+      days_since_epoch_ = date - EPOCH_DATE;
       DCHECK(IsValid());
     }
   }
@@ -85,19 +85,6 @@ int DateValue::Format(const DateTimeFormatContext& dt_ctx, int len, char* buff)
   return DateParser::Format(dt_ctx, *this, len, buff);
 }
 
-bool DateValue::ToYearMonthDay(int* year, int* month, int* day) const {
-  DCHECK(year != nullptr);
-  DCHECK(month != nullptr);
-  DCHECK(day != nullptr);
-  if (UNLIKELY(!IsValid())) return false;
-
-  const cctz::civil_day cd = EPOCH_DATE + days_since_epoch_;
-  *year = cd.year();
-  *month = cd.month();
-  *day = cd.day();
-  return true;
-}
-
 namespace {
 
 inline int32_t CalcFirstDayOfYearSinceEpoch(int year) {
@@ -111,6 +98,10 @@ inline int32_t CalcFirstDayOfYearSinceEpoch(int year) {
       + ((year - EPOCH_YEAR / 400 * 400 + ((m400 != 0) ? 400 - m400 : 0)) / 400 - 1);
 }
 
+inline bool IsLeapYear(int year) {
+  return (year % 4 == 0 && (year % 100 != 0 || year % 400 == 0));
+}
+
 }
 
 bool DateValue::ToYear(int* year) const {
@@ -174,7 +165,46 @@ bool DateValue::ToYear(int* year) const {
   } else {
     *year = first_year;
   }
+  DCHECK(*year >= MIN_YEAR && *year <= MAX_YEAR);
+
+  return true;
+}
+
+bool DateValue::ToYearMonthDay(int* year, int* month, int* day) const {
+  DCHECK(year != nullptr);
+  DCHECK(month != nullptr);
+  DCHECK(day != nullptr);
+  if (UNLIKELY(!IsValid())) return false;
+
+  // Uses the same method to calculate the year as DateValue::ToYear().
+  int tmp = days_since_epoch_ * 400 + 287811200;
+  int first_year = (tmp - 591) / 146097;
+  int last_year = (tmp + 288) / 146097;
+
+  int jan1_dse = CalcFirstDayOfYearSinceEpoch(last_year);
+  if (jan1_dse <= days_since_epoch_) {
+    *year = last_year;
+  } else {
+    *year = first_year;
+    jan1_dse -= IsLeapYear(first_year) ? 366 : 365;
+  }
+  DCHECK(*year >= MIN_YEAR && *year <= MAX_YEAR);
+
+  // Day of year. 0 is used for January 1.
+  int days_since_jan1 = days_since_epoch_ - jan1_dse;
 
+  // Calculate month using month ranges and the average month length.
+  const vector<int>& month_ranges = IsLeapYear(*year) ? LEAP_YEAR_MONTH_RANGES
+                                                      : MONTH_RANGES;
+  int m = static_cast<int>(days_since_jan1 / 30.5);
+  DCHECK(month_ranges[m] <= days_since_jan1);
+
+  *month = (month_ranges[m + 1] <= days_since_jan1) ? m + 2 : m + 1;
+  DCHECK(*month >= 1 && *month <= 12);
+
+  // Calculate day.
+  *day = days_since_jan1 - month_ranges[*month - 1] + 1;
+  DCHECK(*day >= 1 && *day <= 31);
   return true;
 }
 
@@ -184,16 +214,143 @@ int DateValue::WeekDay() const {
   return static_cast<int>(cctz::get_weekday(cd));
 }
 
-DateValue DateValue::AddDays(int days) const {
+int DateValue::DayOfYear() const {
+  if (UNLIKELY(!IsValid())) return -1;
+  const cctz::civil_day cd = EPOCH_DATE + days_since_epoch_;
+  return static_cast<int>(cctz::get_yearday(cd));
+}
+
+int DateValue::WeekOfYear() const {
+  if (UNLIKELY(!IsValid())) return -1;
+  const cctz::civil_day today = EPOCH_DATE + days_since_epoch_;
+
+  cctz::civil_day jan1 = cctz::civil_day(today.year(), 1, 1);
+  cctz::civil_day first_monday;
+  if (cctz::get_weekday(jan1) <= cctz::weekday::thursday) {
+    // Get the previous Monday if 'jan1' is not already a Monday.
+    first_monday = cctz::next_weekday(jan1, cctz::weekday::monday) - 7;
+  } else {
+    // Get the next Monday.
+    first_monday = cctz::next_weekday(jan1, cctz::weekday::monday);
+  }
+
+  cctz::civil_day dec31 = cctz::civil_day(today.year(), 12, 31);
+  cctz::civil_day last_sunday;
+  if (cctz::get_weekday(dec31) >= cctz::weekday::thursday) {
+    // Get the next Sunday if 'dec31' is not already a Sunday.
+    last_sunday = cctz::prev_weekday(dec31, cctz::weekday::sunday) + 7;
+  } else {
+    // Get the previous Sunday.
+    last_sunday = cctz::prev_weekday(dec31, cctz::weekday::sunday);
+  }
+
+  if (UNLIKELY(today.year() == 0 && today < first_monday)) {
+    // 0000-01-01 is Saturday in the proleptic Gregorian calendar.
+    // 0000-01-01 and 0000-01-02 belong to the previous year.
+    return 52;
+  } else if (today >= first_monday && today <= last_sunday) {
+    return (today - first_monday) / 7 + 1;
+  } else if (today > last_sunday) {
+    return 1;
+  } else {
+    // today < first_monday && today.year() > 0
+    cctz::civil_day prev_jan1 = cctz::civil_day(today.year() - 1, 1, 1);
+    cctz::civil_day prev_first_monday;
+    if (cctz::get_weekday(prev_jan1) <= cctz::weekday::thursday) {
+      // Get the previous Monday if 'prev_jan1' is not already a Monday.
+      prev_first_monday = cctz::next_weekday(prev_jan1, cctz::weekday::monday) - 7;
+    } else {
+      // Get the next Monday.
+      prev_first_monday = cctz::next_weekday(prev_jan1, cctz::weekday::monday);
+    }
+    return (today - prev_first_monday) / 7 + 1;
+  }
+}
+
+DateValue DateValue::AddDays(int64_t days) const {
   if (UNLIKELY(!IsValid())) return DateValue();
   return DateValue(days_since_epoch_ + days);
 }
 
+DateValue DateValue::AddMonths(int64_t months, bool keep_last_day) const {
+  if (UNLIKELY(!IsValid())) return DateValue();
+
+  const cctz::civil_day today = EPOCH_DATE + days_since_epoch_;
+  const cctz::civil_month month = cctz::civil_month(today);
+  const cctz::civil_month result_month = month + months;
+  const cctz::civil_day last_day_of_result_month =
+      cctz::civil_day(result_month + 1) - 1;
+
+  if (keep_last_day) {
+    const cctz::civil_day last_day_of_month = cctz::civil_day(month + 1) - 1;
+    if (today == last_day_of_month) {
+      return DateValue(last_day_of_result_month.year(),
+          last_day_of_result_month.month(), last_day_of_result_month.day());
+    }
+  }
+
+  const cctz::civil_day ans_normalized = cctz::civil_day(result_month.year(),
+      result_month.month(), today.day());
+  const cctz::civil_day ans_capped = std::min(ans_normalized, last_day_of_result_month);
+  return DateValue(ans_capped.year(), ans_capped.month(), ans_capped.day());
+}
+
+DateValue DateValue::AddYears(int64_t years) const {
+  if (UNLIKELY(!IsValid())) return DateValue();
+
+  const cctz::civil_day today = EPOCH_DATE + days_since_epoch_;
+  const int64_t result_year = today.year() + years;
+
+  // Feb 29 in leap years requires special attention.
+  if (UNLIKELY(today.month() == 2 && today.day() == 29)) {
+    const cctz::civil_month result_month(result_year, today.month());
+    const cctz::civil_day last_day_of_result_month =
+        cctz::civil_day(result_month + 1) - 1;
+    return DateValue(result_year, last_day_of_result_month.month(),
+        last_day_of_result_month.day());
+  }
+  return DateValue(result_year, today.month(), today.day());
+}
+
 bool DateValue::ToDaysSinceEpoch(int32_t* days) const {
   DCHECK(days != nullptr);
   if (UNLIKELY(!IsValid())) return false;
 
-  *days =  days_since_epoch_;
+  *days = days_since_epoch_;
+  return true;
+}
+
+DateValue DateValue::LastDay() const {
+  if (UNLIKELY(!IsValid())) return DateValue();
+
+  const cctz::civil_day today = EPOCH_DATE + days_since_epoch_;
+  const cctz::civil_month month = cctz::civil_month(today);
+  const cctz::civil_day last_day_of_month = cctz::civil_day(month + 1) - 1;
+  return DateValue(last_day_of_month - EPOCH_DATE);
+}
+
+bool DateValue::MonthsBetween(const DateValue& other, double* months_between) const {
+  DCHECK(months_between != nullptr);
+  if (UNLIKELY(!IsValid() || !other.IsValid())) return false;
+
+  const cctz::civil_day today = EPOCH_DATE + days_since_epoch_;
+  const cctz::civil_month month(today);
+  const cctz::civil_day last_day_of_month = cctz::civil_day(month + 1) - 1;
+
+  const cctz::civil_day other_date = EPOCH_DATE + other.days_since_epoch_;
+  const cctz::civil_month other_month(other_date);
+  const cctz::civil_day last_day_of_other_month = cctz::civil_day(other_month + 1) - 1;
+
+  // If both dates are last days of different months they don't contribute
+  // a fractional value to the number of months, therefore there is no need to
+  // calculate difference in their days.
+  int days_diff = 0;
+  if (today != last_day_of_month || other_date != last_day_of_other_month) {
+    days_diff = today.day() - other_date.day();
+  }
+
+  *months_between = (today.year() - other_date.year()) * 12 +
+      today.month() - other_date.month() + (static_cast<double>(days_diff) / 31.0);
   return true;
 }
 
diff --git a/be/src/runtime/date-value.h b/be/src/runtime/date-value.h
index 606eb7a..b7f2b8a 100644
--- a/be/src/runtime/date-value.h
+++ b/be/src/runtime/date-value.h
@@ -51,6 +51,9 @@ struct DateTimeFormatContext;
 ///   due to its limited range.
 class DateValue {
  public:
+  static const DateValue MIN_DATE;
+  static const DateValue MAX_DATE;
+
   /// Default constructor creates an invalid DateValue instance.
   DateValue() : days_since_epoch_(INVALID_DAYS_SINCE_EPOCH) {
     DCHECK(!IsValid());
@@ -66,7 +69,7 @@ class DateValue {
     }
   }
 
-  DateValue(int year, int month, int day);
+  DateValue(int64_t year, int64_t month, int64_t day);
 
   bool IsValid() const {
     return days_since_epoch_ != INVALID_DAYS_SINCE_EPOCH;
@@ -87,20 +90,52 @@ class DateValue {
   /// Otherwise, return false.
   bool ToYear(int* year) const WARN_UNUSED_RESULT;
 
-  /// If DateValue instance is valid, returns day-of-week in [0, 6]; 0 = Monday and
+  /// If DateValue instance is valid, returns day-of-week in [0, 6] range; 0 = Monday and
   /// 6 = Sunday.
   /// Otherwise, return -1.
   int WeekDay() const;
 
-  /// If this DateValue instance valid, add 'days' to it and return the result.
+  /// If DateValue instance is valid, returns day-of-year in [1, 366] range.
+  /// Otherwise, return -1.
+  int DayOfYear() const;
+
+  /// Returns the week corresponding to his date. Returned value is in the [1, 53] range.
+  /// Weeks start with Monday. Each week's year is the Gregorian year in which the
+  /// Thursday falls.
+  int WeekOfYear() const;
+
+  /// If this DateValue instance valid, add 'days' days to it and return the result.
+  /// Otherwise, return an invalid DateValue instance.
+  DateValue AddDays(int64_t days) const;
+
+  /// If this DateValue instance valid, add 'months' months to it and return the result.
   /// Otherwise, return an invalid DateValue instance.
-  DateValue AddDays(int days) const;
+  /// If 'keep_last_day' is set and this DateValue is the last day of a month, the
+  /// returned DateValue will fall on the last day of the target month too.
+  DateValue AddMonths(int64_t months, bool keep_last_day) const;
+
+  /// If this DateValue instance valid, add 'years' years to it and return the result.
+  /// Otherwise, return an invalid DateValue instance.
+  DateValue AddYears(int64_t years) const;
 
   /// If this DateValue instance is valid, convert it to the number of days since epoch
   /// and return true. Result is placed in 'days'.
   /// Otherwise, return false.
   bool ToDaysSinceEpoch(int32_t* days) const WARN_UNUSED_RESULT;
 
+  /// If this DateValue instance is valid, return DateValue corresponding to the last day
+  /// of the current month.
+  /// Otherwise, return an invalid DateValue instance.
+  DateValue LastDay() const;
+
+  /// If this DateValue and 'other' are both valid, set 'months_between' to the number of
+  /// months between dates and return true. Otherwise return false.
+  /// If this is later than 'other', then the result is positive. If this is earlier than
+  /// 'other', then the result is negative. If this and 'other' are either the same days
+  /// of the month or both last days of months, then the result is always an integer.
+  /// Otherwise calculate the fractional portion of the result based on a 31-day month.
+  bool MonthsBetween(const DateValue& other, double* months_between) const;
+
   /// Returns a DateVal representation in the output variable.
   /// Returns null if the DateValue instance doesn't have a valid date.
   impala_udf::DateVal ToDateVal() const {
@@ -146,16 +181,10 @@ class DateValue {
   bool operator<=(const DateValue& other) const { return !(*this > other); }
   bool operator>=(const DateValue& other) const { return !(*this < other); }
 
-  static const DateValue MIN_DATE;
-  static const DateValue MAX_DATE;
-
  private:
   /// Number of days since 1970.01.01.
   int32_t days_since_epoch_;
 
-  static const int MIN_YEAR;
-  static const int MAX_YEAR;
-
   static const int32_t MIN_DAYS_SINCE_EPOCH;
   static const int32_t MAX_DAYS_SINCE_EPOCH;
   static const int32_t INVALID_DAYS_SINCE_EPOCH = std::numeric_limits<int32_t>::min();
diff --git a/common/function-registry/impala_functions.py b/common/function-registry/impala_functions.py
index 44ee32e..06bf8ce 100644
--- a/common/function-registry/impala_functions.py
+++ b/common/function-registry/impala_functions.py
@@ -269,10 +269,58 @@ visible_functions = [
    '_ZN6impala18TimestampFunctions20UnixAndFromUnixCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
 
   # Date functions
+  [['monthname'], 'STRING', ['DATE'], '_ZN6impala13DateFunctions13LongMonthNameEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['next_day'], 'DATE', ['DATE', 'STRING'], '_ZN6impala13DateFunctions7NextDayEPN10impala_udf15FunctionContextERKNS1_7DateValERKNS1_9StringValE'],
+  [['last_day'], 'DATE', ['DATE'], '_ZN6impala13DateFunctions7LastDayEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['year'], 'INT', ['DATE'], '_ZN6impala13DateFunctions4YearEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['quarter'], 'INT', ['DATE'], '_ZN6impala13DateFunctions7QuarterEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['month'], 'INT', ['DATE'], '_ZN6impala13DateFunctions5MonthEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['dayofweek'], 'INT', ['DATE'], '_ZN6impala13DateFunctions9DayOfWeekEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['day', 'dayofmonth'], 'INT', ['DATE'], '_ZN6impala13DateFunctions10DayOfMonthEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['dayofyear'], 'INT', ['DATE'], '_ZN6impala13DateFunctions9DayOfYearEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['week', 'weekofyear'], 'INT', ['DATE'], '_ZN6impala13DateFunctions10WeekOfYearEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
+  [['dayname'], 'STRING', ['DATE'], '_ZN6impala13DateFunctions11LongDayNameEPN10impala_udf15FunctionContextERKNS1_7DateValE'],
   [['date_trunc'], 'DATE', ['STRING', 'DATE'],
    '_ZN6impala11UdfBuiltins16DateTruncForDateEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_7DateValE',
    '_ZN6impala11UdfBuiltins23DateTruncForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
    '_ZN6impala11UdfBuiltins21DateTruncForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['months_add', 'add_months'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb1EN10impala_udf6IntValELb1EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['months_add', 'add_months'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb1EN10impala_udf9BigIntValELb1EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['months_sub'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb0EN10impala_udf6IntValELb1EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['months_sub'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb0EN10impala_udf9BigIntValELb1EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['years_add'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions11AddSubYearsILb1EN10impala_udf6IntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['years_add'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions11AddSubYearsILb1EN10impala_udf9BigIntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['years_sub'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions11AddSubYearsILb0EN10impala_udf6IntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['years_sub'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions11AddSubYearsILb0EN10impala_udf9BigIntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['weeks_add'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions11AddSubWeeksILb1EN10impala_udf6IntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['weeks_add'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions11AddSubWeeksILb1EN10impala_udf9BigIntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['weeks_sub'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions11AddSubWeeksILb0EN10impala_udf6IntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['weeks_sub'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions11AddSubWeeksILb0EN10impala_udf9BigIntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['days_add', 'date_add', 'adddate'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions10AddSubDaysILb1EN10impala_udf6IntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['days_add', 'date_add', 'adddate'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions10AddSubDaysILb1EN10impala_udf9BigIntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['days_sub', 'date_sub', 'subdate'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions10AddSubDaysILb0EN10impala_udf6IntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['days_sub', 'date_sub', 'subdate'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions10AddSubDaysILb0EN10impala_udf9BigIntValEEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['datediff'], 'INT', ['DATE', 'DATE'], '_ZN6impala13DateFunctions8DateDiffEPN10impala_udf15FunctionContextERKNS1_7DateValES6_'],
+  [['current_date'], 'DATE', [], '_ZN6impala13DateFunctions11CurrentDateEPN10impala_udf15FunctionContextE'],
+  [['date_cmp'], 'INT', ['DATE', 'DATE'], "_ZN6impala13DateFunctions7DateCmpEPN10impala_udf15FunctionContextERKNS1_7DateValES6_"],
+  [['int_months_between'], 'INT', ['DATE', 'DATE'], "_ZN6impala13DateFunctions16IntMonthsBetweenEPN10impala_udf15FunctionContextERKNS1_7DateValES6_"],
+  [['months_between'], 'DOUBLE', ['DATE', 'DATE'], "_ZN6impala13DateFunctions13MonthsBetweenEPN10impala_udf15FunctionContextERKNS1_7DateValES6_"],
 
   # Math builtin functions
   [['pi'], 'DOUBLE', [], 'impala::MathFunctions::Pi'],
@@ -756,6 +804,15 @@ invisible_functions = [
   [['months_sub_interval'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'],
       '_ZN6impala18TimestampFunctions6AddSubILb0EN10impala_udf9BigIntValEN5boost9date_time15months_durationINS4_9gregorian21greg_durations_configEEELb0EEENS2_12TimestampValEPNS2_15FunctionContextERKSA_RKT0_'],
 
+  [['months_add_interval'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb1EN10impala_udf6IntValELb0EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['months_add_interval'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb1EN10impala_udf9BigIntValELb0EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['months_sub_interval'], 'DATE', ['DATE', 'INT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb0EN10impala_udf6IntValELb0EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+  [['months_sub_interval'], 'DATE', ['DATE', 'BIGINT'],
+      '_ZN6impala13DateFunctions12AddSubMonthsILb0EN10impala_udf9BigIntValELb0EEENS2_7DateValEPNS2_15FunctionContextERKS4_RKT0_'],
+
   [['distinctfrom'], 'BOOLEAN', ['BOOLEAN', 'BOOLEAN'], 'impala::Operators::DistinctFrom_BooleanVal_BooleanVal'],
   [['distinctfrom'], 'BOOLEAN', ['TINYINT', 'TINYINT'], 'impala::Operators::DistinctFrom_TinyIntVal_TinyIntVal'],
   [['distinctfrom'], 'BOOLEAN', ['SMALLINT', 'SMALLINT'], 'impala::Operators::DistinctFrom_SmallIntVal_SmallIntVal'],
diff --git a/fe/src/main/java/org/apache/impala/analysis/TimestampArithmeticExpr.java b/fe/src/main/java/org/apache/impala/analysis/TimestampArithmeticExpr.java
index 50e167e..49995cd 100644
--- a/fe/src/main/java/org/apache/impala/analysis/TimestampArithmeticExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/TimestampArithmeticExpr.java
@@ -28,13 +28,14 @@ import org.apache.impala.thrift.TExprNodeType;
 import com.google.common.base.Preconditions;
 
 /**
- * Describes the addition and subtraction of time units from timestamps.
- * Arithmetic expressions on timestamps are syntactic sugar.
+ * Describes the addition and subtraction of time units from timestamps/dates.
+ * Arithmetic expressions on timestamps/dates are syntactic sugar.
  * They are executed as function call exprs in the BE.
+ * TimestampArithmeticExpr is used for both TIMESTAMP and DATE values.
  */
 public class TimestampArithmeticExpr extends Expr {
 
-  // Time units supported in timestamp arithmetic.
+  // Time units supported in timestamp/date arithmetic.
   public static enum TimeUnit {
     YEAR("YEAR"),
     MONTH("MONTH"),
@@ -90,7 +91,7 @@ public class TimestampArithmeticExpr extends Expr {
   }
 
   // C'tor for non-function-call like arithmetic, e.g., 'a + interval b year'.
-  // e1 always refers to the timestamp to be added/subtracted from, and e2
+  // e1 always refers to the timestamp/date to be added/subtracted from, and e2
   // to the time value (even in the interval-first case).
   public TimestampArithmeticExpr(ArithmeticExpr.Operator op, Expr e1, Expr e2,
       String timeUnitIdent, boolean intervalFirst) {
@@ -125,7 +126,7 @@ public class TimestampArithmeticExpr extends Expr {
         op_ = ArithmeticExpr.Operator.SUBTRACT;
       } else {
         throw new AnalysisException("Encountered function name '" + funcName_ +
-            "' in timestamp arithmetic expression '" + toSql() + "'. " +
+            "' in timestamp/date arithmetic expression '" + toSql() + "'. " +
             "Expected function name 'DATE_ADD' or 'DATE_SUB'.");
       }
     }
@@ -133,21 +134,30 @@ public class TimestampArithmeticExpr extends Expr {
     timeUnit_ = TIME_UNITS_MAP.get(timeUnitIdent_.toUpperCase());
     if (timeUnit_ == null) {
       throw new AnalysisException("Invalid time unit '" + timeUnitIdent_ +
-          "' in timestamp arithmetic expression '" + toSql() + "'.");
+          "' in timestamp/date arithmetic expression '" + toSql() + "'.");
     }
 
-    // The first child must return a timestamp or null.
-    if (!getChild(0).getType().isTimestamp() && !getChild(0).getType().isNull()) {
+    // The first child must return a timestamp or date or null.
+    if (!getChild(0).getType().isTimestamp() && !getChild(0).getType().isDate()
+        && !getChild(0).getType().isNull()) {
       throw new AnalysisException("Operand '" + getChild(0).toSql() +
-          "' of timestamp arithmetic expression '" + toSql() + "' returns type '" +
-          getChild(0).getType().toSql() + "'. Expected type 'TIMESTAMP'.");
+          "' of timestamp/date arithmetic expression '" + toSql() + "' returns type '" +
+          getChild(0).getType().toSql() + "'. Expected type 'TIMESTAMP' or 'DATE'.");
+    }
+
+    // If first child returns a date, time unit must be YEAR/MONTH/WEEK/DAY.
+    if (getChild(0).getType().isDate() && timeUnit_ != TimeUnit.YEAR
+        && timeUnit_ != TimeUnit.MONTH && timeUnit_ != TimeUnit.WEEK
+        && timeUnit_ != TimeUnit.DAY) {
+      throw new AnalysisException("'" + timeUnit_ + "' intervals are not allowed in " +
+          "date arithmetic expressions");
     }
 
     // The second child must be an integer type.
     if (!getChild(1).getType().isIntegerType() &&
         !getChild(1).getType().isNull()) {
       throw new AnalysisException("Operand '" + getChild(1).toSql() +
-          "' of timestamp arithmetic expression '" + toSql() + "' returns type '" +
+          "' of timestamp/date arithmetic expression '" + toSql() + "' returns type '" +
           getChild(1).getType().toSql() + "'. Expected an integer type.");
     }
 
@@ -162,7 +172,8 @@ public class TimestampArithmeticExpr extends Expr {
     castForFunctionCall(false, analyzer.isDecimalV2());
 
     Preconditions.checkNotNull(fn_);
-    Preconditions.checkState(fn_.getReturnType().isTimestamp());
+    Preconditions.checkState(fn_.getReturnType().isTimestamp()
+        || fn_.getReturnType().isDate());
     type_ = fn_.getReturnType();
   }
 
diff --git a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
index 2bb4238..23910f4 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
@@ -1683,7 +1683,7 @@ public class AnalyzeExprsTest extends AnalyzerTest {
   }
 
   /**
-   * We have three variants of timestamp arithmetic exprs, as in MySQL:
+   * We have three variants of timestamp/date arithmetic exprs, as in MySQL:
    * http://dev.mysql.com/doc/refman/5.5/en/date-and-time-functions.html
    * (section #function_date-add)
    * 1. Non-function-call like version, e.g., 'a + interval b timeunit'
@@ -1723,77 +1723,79 @@ public class AnalyzeExprsTest extends AnalyzerTest {
 
     // First operand does not return a timestamp. Non-function-call like version.
     AnalysisError("select float_col + interval 10 years from functional.alltypes",
-        "Operand 'float_col' of timestamp arithmetic expression " +
+        "Operand 'float_col' of timestamp/date arithmetic expression " +
         "'float_col + INTERVAL 10 years' returns type 'FLOAT'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
     AnalysisError("select string_col + interval 10 years from functional.alltypes",
-        "Operand 'string_col' of timestamp arithmetic expression " +
+        "Operand 'string_col' of timestamp/date arithmetic expression " +
         "'string_col + INTERVAL 10 years' returns type 'STRING'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
     AnalysisError(
         "select int_struct_col + interval 10 years from functional.allcomplextypes",
-        "Operand 'int_struct_col' of timestamp arithmetic expression " +
+        "Operand 'int_struct_col' of timestamp/date arithmetic expression " +
         "'int_struct_col + INTERVAL 10 years' returns type 'STRUCT<f1:INT,f2:INT>'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
     // Reversed interval and timestamp using addition.
     AnalysisError("select interval 10 years + float_col from functional.alltypes",
-        "Operand 'float_col' of timestamp arithmetic expression " +
+        "Operand 'float_col' of timestamp/date arithmetic expression " +
         "'INTERVAL 10 years + float_col' returns type 'FLOAT'. " +
-        "Expected type 'TIMESTAMP'");
+        "Expected type 'TIMESTAMP' or 'DATE'");
     AnalysisError("select interval 10 years + string_col from functional.alltypes",
-        "Operand 'string_col' of timestamp arithmetic expression " +
+        "Operand 'string_col' of timestamp/date arithmetic expression " +
         "'INTERVAL 10 years + string_col' returns type 'STRING'. " +
-        "Expected type 'TIMESTAMP'");
+        "Expected type 'TIMESTAMP' or 'DATE'");
     AnalysisError(
         "select interval 10 years + int_array_col from functional.allcomplextypes",
-        "Operand 'int_array_col' of timestamp arithmetic expression " +
+        "Operand 'int_array_col' of timestamp/date arithmetic expression " +
         "'INTERVAL 10 years + int_array_col' returns type 'ARRAY<INT>'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
     // First operand does not return a timestamp. Function-call like version.
     AnalysisError("select date_add(float_col, interval 10 years) " +
         "from functional.alltypes",
-        "Operand 'float_col' of timestamp arithmetic expression " +
+        "Operand 'float_col' of timestamp/date arithmetic expression " +
         "'DATE_ADD(float_col, INTERVAL 10 years)' returns type 'FLOAT'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
     AnalysisError("select date_add(string_col, interval 10 years) " +
         "from functional.alltypes",
-        "Operand 'string_col' of timestamp arithmetic expression " +
+        "Operand 'string_col' of timestamp/date arithmetic expression " +
         "'DATE_ADD(string_col, INTERVAL 10 years)' returns type 'STRING'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
     AnalysisError("select date_add(int_map_col, interval 10 years) " +
         "from functional.allcomplextypes",
-        "Operand 'int_map_col' of timestamp arithmetic expression " +
+        "Operand 'int_map_col' of timestamp/date arithmetic expression " +
         "'DATE_ADD(int_map_col, INTERVAL 10 years)' returns type 'MAP<STRING,INT>'. " +
-        "Expected type 'TIMESTAMP'.");
+        "Expected type 'TIMESTAMP' or 'DATE'.");
 
     // Second operand is not compatible with a fixed-point type.
     // Non-function-call like version.
     AnalysisError("select timestamp_col + interval 5.2 years from functional.alltypes",
-        "Operand '5.2' of timestamp arithmetic expression " +
+        "Operand '5.2' of timestamp/date arithmetic expression " +
         "'timestamp_col + INTERVAL 5.2 years' returns type 'DECIMAL(2,1)'. " +
         "Expected an integer type.");
     AnalysisError("select cast(0 as timestamp) + interval int_array_col years " +
         "from functional.allcomplextypes",
-        "Operand 'int_array_col' of timestamp arithmetic expression " +
+        "Operand 'int_array_col' of timestamp/date arithmetic expression " +
         "'CAST(0 AS TIMESTAMP) + INTERVAL int_array_col years' " +
         "returns type 'ARRAY<INT>'. Expected an integer type.");
 
     // No implicit cast from STRING to integer types.
     AnalysisError("select timestamp_col + interval '10' years from functional.alltypes",
-                  "Operand ''10'' of timestamp arithmetic expression 'timestamp_col + " +
-                  "INTERVAL '10' years' returns type 'STRING'. " +
+                  "Operand ''10'' of timestamp/date arithmetic expression " +
+                  "'timestamp_col + INTERVAL '10' years' returns type 'STRING'. " +
                   "Expected an integer type.");
     AnalysisError("select date_add(timestamp_col, interval '10' years) " +
-                  "from functional.alltypes", "Operand ''10'' of timestamp arithmetic " +
-                  "expression 'DATE_ADD(timestamp_col, INTERVAL '10' years)' returns " +
-                  "type 'STRING'. Expected an integer type.");
+                  "from functional.alltypes",
+                  "Operand ''10'' of timestamp/date " +
+                  "arithmetic expression " +
+                  "'DATE_ADD(timestamp_col, INTERVAL '10' years)' returns type " +
+                  "'STRING'. Expected an integer type.");
 
     // Cast from STRING to INT.
     AnalyzesOk("select timestamp_col + interval cast('10' as int) years " +
         "from functional.alltypes");
     // Reversed interval and timestamp using addition.
     AnalysisError("select interval 5.2 years + timestamp_col from functional.alltypes",
-        "Operand '5.2' of timestamp arithmetic expression " +
+        "Operand '5.2' of timestamp/date arithmetic expression " +
         "'INTERVAL 5.2 years + timestamp_col' returns type 'DECIMAL(2,1)'. " +
         "Expected an integer type.");
     // Cast from STRING to INT.
@@ -1802,7 +1804,7 @@ public class AnalyzeExprsTest extends AnalyzerTest {
     // Second operand is not compatible with type INT. Function-call like version.
     AnalysisError("select date_add(timestamp_col, interval 5.2 years) " +
         "from functional.alltypes",
-        "Operand '5.2' of timestamp arithmetic expression " +
+        "Operand '5.2' of timestamp/date arithmetic expression " +
         "'DATE_ADD(timestamp_col, INTERVAL 5.2 years)' returns type 'DECIMAL(2,1)'. " +
         "Expected an integer type.");
     // Cast from STRING to INT.
@@ -1811,23 +1813,23 @@ public class AnalyzeExprsTest extends AnalyzerTest {
 
     // Invalid time unit. Non-function-call like version.
     AnalysisError("select timestamp_col + interval 10 error from functional.alltypes",
-        "Invalid time unit 'error' in timestamp arithmetic expression " +
+        "Invalid time unit 'error' in timestamp/date arithmetic expression " +
          "'timestamp_col + INTERVAL 10 error'.");
     AnalysisError("select timestamp_col - interval 10 error from functional.alltypes",
-        "Invalid time unit 'error' in timestamp arithmetic expression " +
+        "Invalid time unit 'error' in timestamp/date arithmetic expression " +
          "'timestamp_col - INTERVAL 10 error'.");
     // Reversed interval and timestamp using addition.
     AnalysisError("select interval 10 error + timestamp_col from functional.alltypes",
-        "Invalid time unit 'error' in timestamp arithmetic expression " +
+        "Invalid time unit 'error' in timestamp/date arithmetic expression " +
         "'INTERVAL 10 error + timestamp_col'.");
     // Invalid time unit. Function-call like version.
     AnalysisError("select date_add(timestamp_col, interval 10 error) " +
         "from functional.alltypes",
-        "Invalid time unit 'error' in timestamp arithmetic expression " +
+        "Invalid time unit 'error' in timestamp/date arithmetic expression " +
         "'DATE_ADD(timestamp_col, INTERVAL 10 error)'.");
     AnalysisError("select date_sub(timestamp_col, interval 10 error) " +
         "from functional.alltypes",
-        "Invalid time unit 'error' in timestamp arithmetic expression " +
+        "Invalid time unit 'error' in timestamp/date arithmetic expression " +
         "'DATE_SUB(timestamp_col, INTERVAL 10 error)'.");
   }
 
@@ -3064,5 +3066,15 @@ public class AnalyzeExprsTest extends AnalyzerTest {
     Assert.assertEquals(Type.TIMESTAMP, foundFn.getArgs()[1]);
     Assert.assertEquals(Type.TIMESTAMP, foundFn.getArgs()[2]);
 
+    // String is matched non-strictly to timestamp if there's no string overload but both
+    // timestamp and date overloads are available.
+    FunctionName yearFnName = new FunctionName(BuiltinsDb.NAME, "year");
+    Function yearStringFn = new Function(yearFnName,
+        new Type[] {ScalarType.STRING}, Type.INT, false);
+    foundFn = db.getFunction(yearStringFn, CompareMode.IS_SUPERTYPE_OF);
+    Assert.assertNull(foundFn);
+    foundFn = db.getFunction(yearStringFn, CompareMode.IS_NONSTRICT_SUPERTYPE_OF);
+    Assert.assertNotNull(foundFn);
+    Assert.assertEquals(Type.TIMESTAMP, foundFn.getArgs()[0]);
   }
 }