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/05 16:06:55 UTC

[impala] 04/06: IMPALA-7369: part 1: Implement TRUNC, DATE_TRUNC, EXTRACT, DATE_PART 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 f0678b06e64b57e0b26ca73577444e1941925dbc
Author: Attila Jeges <at...@cloudera.com>
AuthorDate: Thu May 16 17:38:22 2019 +0200

    IMPALA-7369: part 1: Implement TRUNC, DATE_TRUNC, EXTRACT, DATE_PART functions for DATE
    
    These functions are somewhat similar in that each of them takes a DATE
    argument and a time unit to work with.
    
    They work identically to the corresponding TIMESTAMP functions. The
    only difference is that the DATE functions don't accept time-of-day
    units.
    
    TRUNC(DATE d, STRING unit)
    Truncates a DATE value to the specified time unit. The 'unit' argument
    is case insensitive. This argument string can be one of:
      SYYYY, YYYY, YEAR, SYEAR, YYY, YY, Y: Year.
      Q: Quarter.
      MONTH, MON, MM, RM: Month.
      DDD, DD, J: Day.
      DAY, DY, D: Starting day (Monday) of the week.
      WW: Truncates to the most recent date, no later than 'd', which is
          on the same day of the week as the first day of year.
      W: Truncates to the most recent date, no later than 'd', which is on
         the same day of the week as the first day of month.
    
    The impelementation mirrors Impala's TRUNC(TIMESTAMP ts, STRING unit)
    function. Hive and Oracle SQL have a similar function too.
    Reference:
    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions201.htm
    .
    
    DATE_TRUNC(STRING unit, DATE d)
    Truncates a DATE value to the specified precision. The 'unit' argument
    is case insensitive. This argument string can be one of: DAY, WEEK,
    MONTH, YEAR, DECADE, CENTURY, MILLENNIUM.
    
    The implementation mirrors Impala's DATE_TRUNC(STRING unit,
    TIMESTAMP ts) function. Vertica has a similar function too.
    Reference:
    https://my.vertica.com/docs/8.1.x/HTML/index.htm#Authoring/
        SQLReferenceManual/Functions/Date-Time/DATE_TRUNC.htm
    .
    
    EXTRACT(DATE d, STRING unit), EXTRACT(unit FROM DATE d)
    Returns one of the numeric date fields from a DATE value. The 'unit'
    string can be one of YEAR, QUARTER, MONTH, DAY. This argument value is
    case-insensitive.
    
    The implementation mirrors that Impala's EXTRACT(TIMESTAMP ts,
    STRING unit). Hive and Oracle SQL have a similar function too.
    Reference:
    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
    .
    
    DATE_PART(STRING unit, DATE date)
    Similar to EXTRACT(), with the argument order reversed. Supports the
    same date units as EXTRACT().
    
    The implementation mirrors Impala's DATE_PART(STRING unit,
    TIMESTAMP ts) function.
    
    Change-Id: I843358a45eb5faa2c134994600546fc1d0a797c8
    Reviewed-on: http://gerrit.cloudera.org:8080/13363
    Reviewed-by: Impala Public Jenkins <im...@cloudera.com>
    Tested-by: Impala Public Jenkins <im...@cloudera.com>
---
 be/src/benchmarks/CMakeLists.txt                   |   1 +
 be/src/benchmarks/date-benchmark.cc                | 118 +++++
 be/src/exprs/expr-test.cc                          | 252 +++++++++
 be/src/exprs/udf-builtins-ir.cc                    | 295 +++++------
 be/src/exprs/udf-builtins.cc                       | 567 ++++++++++++++++++---
 be/src/exprs/udf-builtins.h                        | 160 ++++--
 be/src/runtime/date-test.cc                        |  87 ++++
 be/src/runtime/date-value.cc                       |  94 +++-
 be/src/runtime/date-value.h                        |  14 +
 common/function-registry/impala_functions.py       |  42 +-
 .../apache/impala/analysis/ExtractFromExpr.java    |  32 +-
 .../apache/impala/analysis/AnalyzeExprsTest.java   |  11 +-
 .../queries/QueryTest/hdfs-partitions.test         |   2 +-
 13 files changed, 1373 insertions(+), 302 deletions(-)

diff --git a/be/src/benchmarks/CMakeLists.txt b/be/src/benchmarks/CMakeLists.txt
index dfe3d23..83e3504 100644
--- a/be/src/benchmarks/CMakeLists.txt
+++ b/be/src/benchmarks/CMakeLists.txt
@@ -59,5 +59,6 @@ ADD_BE_BENCHMARK(string-search-benchmark)
 ADD_BE_BENCHMARK(thread-create-benchmark)
 ADD_BE_BENCHMARK(tuple-layout-benchmark)
 ADD_BE_BENCHMARK(convert-timestamp-benchmark)
+ADD_BE_BENCHMARK(date-benchmark)
 
 target_link_libraries(hash-benchmark Experiments)
diff --git a/be/src/benchmarks/date-benchmark.cc b/be/src/benchmarks/date-benchmark.cc
new file mode 100644
index 0000000..c0f176c
--- /dev/null
+++ b/be/src/benchmarks/date-benchmark.cc
@@ -0,0 +1,118 @@
+// 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 <iostream>
+#include <vector>
+
+#include "cctz/civil_time.h"
+#include "gutil/basictypes.h"
+#include "runtime/date-value.h"
+#include "util/benchmark.h"
+#include "util/cpu-info.h"
+
+#include "common/names.h"
+
+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
+//                                                                   (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
+
+class TestData {
+public:
+  void AddRange(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);
+      DCHECK(dv.IsValid());
+    }
+    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;
+    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));
+      }
+    }
+  }
+
+  void TestToYear(int batch_size) {
+    DCHECK(date_.size() == to_y_result_year_.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]));
+      }
+    }
+  }
+
+  bool CheckResults() {
+    DCHECK(to_ymd_result_year_.size() == to_y_result_year_.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;
+        ok = false;
+      }
+    }
+    return ok;
+  }
+
+private:
+  vector<DateValue> date_;
+  vector<int> to_ymd_result_year_;
+  vector<int> to_y_result_year_;
+};
+
+void TestToYearMonthDay(int batch_size, void* d) {
+  TestData* data = reinterpret_cast<TestData*>(d);
+  data->TestToYearMonthDay(batch_size);
+}
+
+void TestToYear(int batch_size, void* d) {
+  TestData* data = reinterpret_cast<TestData*>(d);
+  data->TestToYear(batch_size);
+}
+
+int main(int argc, char* argv[]) {
+  CpuInfo::Init();
+  cout << Benchmark::GetMachineInfo() << endl;
+
+  TestData data;
+  data.AddRange(DateValue(1965, 1, 1), DateValue(2020, 12, 31));
+
+  // Benchmark DateValue::ToYearMonthDay() vs DateValue::ToYear()
+  Benchmark suite("ToYear");
+  suite.AddBenchmark("TestToYearMonthDay", TestToYearMonthDay, &data);
+  suite.AddBenchmark("TestToYear", TestToYear, &data);
+  cout << suite.Measure();
+
+  return data.CheckResults() ? 0 : 1;
+}
diff --git a/be/src/exprs/expr-test.cc b/be/src/exprs/expr-test.cc
index f8486b9..f89d953 100644
--- a/be/src/exprs/expr-test.cc
+++ b/be/src/exprs/expr-test.cc
@@ -7421,6 +7421,258 @@ TEST_P(ExprTest, TimestampFunctions) {
   TestIsNull("unix_micros_to_utc_timestamp(253402300800000000)", TYPE_TIMESTAMP);
 }
 
+TEST_P(ExprTest, TruncForDateTest) {
+  // trunc(date, string unit)
+  // Truncate date to year
+  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));
+  }
+  TestDateValue("trunc(date'2000-01-01', 'Y')", DateValue(2000, 1, 1));
+
+  // Truncate date to quarter
+  TestDateValue("trunc(date'2000-01-01', 'Q')", DateValue(2000, 1, 1));
+  TestDateValue("trunc(date'2000-02-01', 'Q')", DateValue(2000, 1, 1));
+  TestDateValue("trunc(date'2000-03-01', 'Q')", DateValue(2000, 1, 1));
+  TestDateValue("trunc(date'2000-04-01', 'Q')", DateValue(2000, 4, 1));
+  TestDateValue("trunc(date'2000-05-01', 'Q')", DateValue(2000, 4, 1));
+  TestDateValue("trunc(date'2000-06-01', 'Q')", DateValue(2000, 4, 1));
+  TestDateValue("trunc(date'2000-07-01', 'Q')", DateValue(2000, 7, 1));
+  TestDateValue("trunc(date'2000-08-01', 'Q')", DateValue(2000, 7, 1));
+  TestDateValue("trunc(date'2000-09-01', 'Q')", DateValue(2000, 7, 1));
+  TestDateValue("trunc(date'2000-10-01', 'Q')", DateValue(2000, 10, 1));
+  TestDateValue("trunc(date'2000-11-01', 'Q')", DateValue(2000, 10, 1));
+  TestDateValue("trunc(date'2000-12-01', 'Q')", DateValue(2000, 10, 1));
+
+  // Truncate date to month
+  for (const string& unit: { "MONTH", "MON", "MM", "RM" }) {
+    const string expr = "trunc(date'2001-02-05', '" + unit + "')";
+    TestDateValue(expr, DateValue(2001, 2, 1));
+  }
+  TestDateValue("trunc(date'2001-01-01', 'MM')", DateValue(2001, 1, 1));
+  TestDateValue("trunc(date'2001-12-29', 'MM')", DateValue(2001, 12, 1));
+
+  // Same day of the week as the first day of the year
+  TestDateValue("trunc(date'2014-01-07', 'WW')", DateValue(2014, 1, 1));
+  TestDateValue("trunc(date'2014-01-08', 'WW')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-09', 'WW')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-14', 'WW')", DateValue(2014, 1, 8));
+
+  // Same day of the week as the first day of the month
+  TestDateValue("trunc(date'2014-01-07', 'W')", DateValue(2014, 1, 1));
+  TestDateValue("trunc(date'2014-01-08', 'W')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-09', 'W')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-01-14', 'W')", DateValue(2014, 1, 8));
+  TestDateValue("trunc(date'2014-02-01', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-02', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-03', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-07', 'W')", DateValue(2014, 2, 1));
+  TestDateValue("trunc(date'2014-02-08', 'W')", DateValue(2014, 2, 8));
+  TestDateValue("trunc(date'2014-02-24', 'W')", DateValue(2014, 2, 22));
+
+  // Truncate to day, i.e. leave the date intact
+  for (const string& unit: { "DDD", "DD", "J" }) {
+    const string expr = "trunc(date'2014-01-08', '" + unit + "')";
+    TestDateValue(expr, DateValue(2014, 1, 8));
+  }
+
+  // Truncate date to starting day of the week
+  for (const string& unit: { "DAY", "DY", "D" }) {
+    const string expr = "trunc(date'2012-09-10', '" + unit + "')";
+    TestDateValue(expr, DateValue(2012, 9, 10));
+  }
+  TestDateValue("trunc(date'2012-09-11', 'D')", DateValue(2012, 9, 10));
+  TestDateValue("trunc(date'2012-09-12', 'D')", DateValue(2012, 9, 10));
+  TestDateValue("trunc(date'2012-09-16', 'D')", DateValue(2012, 9, 10));
+
+  // Test upper limit
+  TestDateValue("trunc(date'9999-12-31', 'YYYY')", DateValue(9999, 1, 1));
+  TestDateValue("trunc(date'9999-12-31', 'Q')", DateValue(9999, 10, 1));
+  TestDateValue("trunc(date'9999-12-31', 'MONTH')", DateValue(9999, 12, 1));
+  TestDateValue("trunc(date'9999-12-31', 'W')", DateValue(9999, 12, 29));
+  TestDateValue("trunc(date'9999-12-31', 'WW')", DateValue(9999, 12, 31));
+  TestDateValue("trunc(date'9999-12-31', 'DDD')", DateValue(9999, 12, 31));
+  TestDateValue("trunc(date'9999-12-31', 'DAY')", DateValue(9999, 12, 27));
+
+  // Test lower limit
+  TestDateValue("trunc(date'0000-01-01', 'YYYY')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-01', 'Q')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-03-31', 'Q')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-01', 'MONTH')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-01', 'W')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-07', 'W')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-08', 'W')", DateValue(0, 1, 8));
+  TestDateValue("trunc(date'0000-01-01', 'WW')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-07', 'WW')", DateValue(0, 1, 1));
+  TestDateValue("trunc(date'0000-01-08', 'WW')", DateValue(0, 1, 8));
+  TestDateValue("trunc(date'0000-01-04', 'DAY')", DateValue(0, 1, 3));
+  TestDateValue("trunc(date'0000-01-03', 'DAY')", DateValue(0, 1, 3));
+  TestIsNull("trunc(date'0000-01-02', 'DAY')", TYPE_DATE);
+  TestIsNull("trunc(date'0000-01-01', 'DAY')", TYPE_DATE);
+
+  // Truncating date to hour or minute returns an error
+  for (const string& unit: { "HH", "HH12", "HH24", "MI" }) {
+    const string expr = "trunc(date'2012-09-10', '" + unit + "')";
+    TestNonOkStatus(expr);  // Unsupported Truncate Unit
+  }
+
+  // Invalid trunc unit
+  for (const string& unit: { "MIN", "XXYYZZ", "" }) {
+    const string expr = "trunc(date'2012-09-10', '" + unit + "')";
+    TestNonOkStatus(expr);  // Invalid Truncate Unit
+  }
+
+  TestIsNull("trunc(cast(NULL as date), 'DDD')", TYPE_DATE);
+  TestNonOkStatus("trunc(cast(NULL as date), NULL)");
+}
+
+TEST_P(ExprTest, DateTruncForDateTest) {
+  TestDateValue("date_trunc('MILLENNIUM', date '2016-05-08')", DateValue(2001, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '3000-12-31')", DateValue(2001, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '3001-01-01')", DateValue(3001, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '2016-05-08')", DateValue(2001, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '2116-05-08')", DateValue(2101, 1, 1));
+  TestDateValue("date_trunc('DECADE', date '2116-05-08')", DateValue(2110, 1, 1));
+  TestDateValue("date_trunc('YEAR', date '2016-05-08')", DateValue(2016, 1, 1));
+  TestDateValue("date_trunc('MONTH', date '2016-05-08')", DateValue(2016, 5, 1));
+  TestDateValue("date_trunc('WEEK', date '2116-05-08')", DateValue(2116, 5, 4));
+  TestDateValue("date_trunc('WEEK', date '2017-01-01')", DateValue(2016,12,26));
+  TestDateValue("date_trunc('WEEK', date '2017-01-02')", DateValue(2017, 1, 2));
+  TestDateValue("date_trunc('WEEK', date '2017-01-07')", DateValue(2017, 1, 2));
+  TestDateValue("date_trunc('WEEK', date '2017-01-08')", DateValue(2017, 1, 2));
+  TestDateValue("date_trunc('WEEK', date '2017-01-09')", DateValue(2017, 1, 9));
+  TestDateValue("date_trunc('DAY', date '1416-05-08')", DateValue(1416, 5, 8));
+
+  // Test upper limit
+  TestDateValue("date_trunc('MILLENNIUM', date '9999-12-31')", DateValue(9001, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '9999-12-31')", DateValue(9901, 1, 1));
+  TestDateValue("date_trunc('DECADE', date '9999-12-31')", DateValue(9990, 1, 1));
+  TestDateValue("date_trunc('YEAR', date '9999-12-31')", DateValue(9999, 1, 1));
+  TestDateValue("date_trunc('MONTH', date '9999-12-31')", DateValue(9999, 12, 1));
+  TestDateValue("date_trunc('WEEK', date '9999-12-31')", DateValue(9999, 12, 27));
+  TestDateValue("date_trunc('DAY', date '9999-12-31')", DateValue(9999, 12, 31));
+
+  // Test lower limit for millennium
+  TestDateValue("date_trunc('MILLENNIUM', date '1001-01-01')", DateValue(1001, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '1000-01-01')", DateValue(1, 1, 1));
+  TestDateValue("date_trunc('MILLENNIUM', date '0001-01-01')", DateValue(1, 1, 1));
+  TestIsNull("date_trunc('MILLENNIUM', date '0000-01-01')", TYPE_DATE);
+
+  // Test lower limit for century
+  TestDateValue("date_trunc('CENTURY', date '0101-01-01')", DateValue(101, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '0100-01-01')", DateValue(1, 1, 1));
+  TestDateValue("date_trunc('CENTURY', date '0001-01-01')", DateValue(1, 1, 1));
+  TestIsNull("date_trunc('CENTURY', date '0000-01-01')", TYPE_DATE);
+
+  // Test lower limit for decade
+  TestDateValue("date_trunc('DECADE', date '0001-01-01')", DateValue(0, 1, 1));
+  TestDateValue("date_trunc('DECADE', date '0000-01-01')", DateValue(0, 1, 1));
+
+  // Test lower limit for year, month, day
+  TestDateValue("date_trunc('YEAR', date '0000-01-01')", DateValue(0, 1, 1));
+  TestDateValue("date_trunc('MONTH', date '0000-01-01')", DateValue(0, 1, 1));
+  TestDateValue("date_trunc('DAY', date '0000-01-01')", DateValue(0, 1, 1));
+
+  // Test lower limit for week
+  TestDateValue("date_trunc('WEEK', date '0000-01-09')", DateValue(0, 1, 3));
+  TestDateValue("date_trunc('WEEK', date '0000-01-03')", DateValue(0, 1, 3));
+  TestIsNull("date_trunc('WEEK', date '0000-01-02')", TYPE_DATE);
+  TestIsNull("date_trunc('WEEK', date '0000-01-01')", TYPE_DATE);
+
+  // Test invalid input.
+  // Truncating date to hour or minute returns an error
+  for (const string& unit: { "HOUR", "MINUTE", "SECOND", "MILLISECONDS",
+      "MICROSECONDS" }) {
+    const string expr = "date_trunc('" + unit + "', date '2012-09-10')";
+    TestNonOkStatus(expr);  // Unsupported Date Truncate Unit
+  }
+
+  // Invalid trunc unit
+  for (const string& unit: { "YEARR", "XXYYZZ", "" }) {
+    const string expr = "date_trunc('" + unit + "', date '2012-09-10')";
+    TestNonOkStatus(expr);  // Invalid Date Truncate Unit
+  }
+
+  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) {
+  // extract as a regular function
+  TestValue("extract(date '2006-05-12', 'YEAR')", TYPE_BIGINT, 2006);
+  TestValue("extract(date '2006-05-12', 'quarter')", TYPE_BIGINT, 2);
+  TestValue("extract(date '2006-05-12', 'MoNTH')", TYPE_BIGINT, 5);
+  TestValue("extract(date '2006-05-12', 'DaY')", TYPE_BIGINT, 12);
+
+  // extract using FROM keyword
+  TestValue("extract(year from date '2006-05-12')", TYPE_BIGINT, 2006);
+  TestValue("extract(QUARTER from date '2006-05-12')", TYPE_BIGINT, 2);
+  TestValue("extract(mOnTh from date '2006-05-12')", TYPE_BIGINT, 5);
+  TestValue("extract(dAy from date '2006-05-12')", TYPE_BIGINT, 12);
+
+  // Test upper limit
+  TestValue("extract(date '9999-12-31', 'YEAR')", TYPE_BIGINT, 9999);
+  TestValue("extract(quarter from date '9999-12-31')", TYPE_BIGINT, 4);
+  TestValue("extract(date '9999-12-31', 'month')", TYPE_BIGINT, 12);
+  TestValue("extract(DAY from date '9999-12-31')", TYPE_BIGINT, 31);
+
+  // Test lower limit
+  TestValue("extract(date '0000-01-01', 'YEAR')", TYPE_BIGINT, 0);
+  TestValue("extract(quarter from date '0000-01-01')", TYPE_BIGINT, 1);
+  TestValue("extract(date '0000-01-01', 'month')", TYPE_BIGINT, 1);
+  TestValue("extract(DAY from date '0000-01-01')", TYPE_BIGINT, 1);
+
+  // Time of day extract fields are not supported
+  for (const string& field: { "MINUTE", "SECOND", "MILLISECOND", "EPOCH" }) {
+    const string expr = "extract(date '2012-09-10', '" + field + "')";
+    TestNonOkStatus(expr);  // Unsupported Extract Field
+  }
+
+  // Invalid extract fields
+  for (const string& field: { "foo", "SSECOND", "" }) {
+    const string expr = "extract(date '2012-09-10', '" + field + "')";
+    TestNonOkStatus(expr);  // 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);
+  TestValue("date_part('QuarTer', date '2006-05-12')", TYPE_BIGINT, 2);
+  TestValue("date_part('Month', date '2006-05-12')", TYPE_BIGINT, 5);
+  TestValue("date_part('Day', date '2006-05-12')", TYPE_BIGINT, 12);
+
+  // Test upper limit
+  TestValue("date_part('YEAR', date '9999-12-31')", TYPE_BIGINT, 9999);
+  TestValue("date_part('QUARTER', '9999-12-31')", TYPE_BIGINT, 4);
+  TestValue("date_part('month', date '9999-12-31')", TYPE_BIGINT, 12);
+  TestValue("date_part('DAY', date '9999-12-31')", TYPE_BIGINT, 31);
+
+  // Test lower limit
+  TestValue("date_part('year', date '0000-01-01')", TYPE_BIGINT, 0);
+  TestValue("date_part('quarter', date '0000-01-01')", TYPE_BIGINT, 1);
+  TestValue("date_part('MONTH', date '0000-01-01')", TYPE_BIGINT, 1);
+  TestValue("date_part('DAY', date '0000-01-01')", TYPE_BIGINT, 1);
+
+  // Time of day extract fields are not supported
+  for (const string& field: { "MINUTE", "SECOND", "MILLISECOND", "EPOCH" }) {
+    const string expr = "date_part('" + field + "', date '2012-09-10')";
+    // Unsupported Date Part Field
+    TestNonOkStatus(expr);
+  }
+
+  // Invalid extract fields
+  for (const string& field: { "foo", "SSECOND", "" }) {
+    const string expr = "date_part('" + field + "', date '2012-09-10')";
+    TestNonOkStatus(expr);  // 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, ConditionalFunctions) {
   // If first param evaluates to true, should return second parameter,
   // false or NULL should return the third.
diff --git a/be/src/exprs/udf-builtins-ir.cc b/be/src/exprs/udf-builtins-ir.cc
index c870fae..05694b7 100644
--- a/be/src/exprs/udf-builtins-ir.cc
+++ b/be/src/exprs/udf-builtins-ir.cc
@@ -25,20 +25,12 @@
 #include <string>
 
 #include "gen-cpp/Exprs_types.h"
-#include "gutil/walltime.h"
 #include "runtime/runtime-state.h"
-#include "runtime/timestamp-value.h"
 #include "udf/udf-internal.h"
-#include "util/bit-util.h"
 
 #include "common/names.h"
 
-using boost::gregorian::date;
-using boost::gregorian::date_duration;
-using boost::posix_time::ptime;
-using boost::posix_time::time_duration;
 using namespace impala;
-using namespace strings;
 
 DoubleVal UdfBuiltins::Abs(FunctionContext* context, const DoubleVal& v) {
   if (v.is_null) return v;
@@ -104,161 +96,6 @@ BooleanVal UdfBuiltins::IsInf(FunctionContext* context, const DoubleVal& val) {
   return BooleanVal(std::isinf(val.val));
 }
 
-TimestampVal UdfBuiltins::Trunc(FunctionContext* context, const TimestampVal& tv,
-    const StringVal &unit_str) {
-  return TruncImpl(context, tv, unit_str);
-}
-
-TimestampVal UdfBuiltins::DateTrunc(
-    FunctionContext* context, const StringVal& unit_str, const TimestampVal& tv) {
-  return DateTruncImpl(context, tv, unit_str);
-}
-
-static int64_t ExtractMillisecond(const time_duration& time) {
-  // Fractional seconds are nanoseconds because Boost is configured
-  // to use nanoseconds precision
-  return time.fractional_seconds() / (NANOS_PER_MICRO * MICROS_PER_MILLI)
-       + time.seconds() * MILLIS_PER_SEC;
-}
-
-// Maps the user facing name of a unit to a TExtractField
-// Returns the TExtractField for the given unit
-TExtractField::type StrToExtractField(FunctionContext* ctx, const StringVal& unit_str) {
-  StringVal unit = UdfBuiltins::Lower(ctx, unit_str);
-  if (UNLIKELY(unit.is_null)) return TExtractField::INVALID_FIELD;
-  if (unit == "year") return TExtractField::YEAR;
-  if (unit == "quarter") return TExtractField::QUARTER;
-  if (unit == "month") return TExtractField::MONTH;
-  if (unit == "day") return TExtractField::DAY;
-  if (unit == "hour") return TExtractField::HOUR;
-  if (unit == "minute") return TExtractField::MINUTE;
-  if (unit == "second") return TExtractField::SECOND;
-  if (unit == "millisecond") return TExtractField::MILLISECOND;
-  if (unit == "epoch") return TExtractField::EPOCH;
-  return TExtractField::INVALID_FIELD;
-}
-
-BigIntVal UdfBuiltins::Extract(FunctionContext* context, const StringVal& unit_str,
-    const TimestampVal& tv) {
-  // resolve extract_field using the prepared state if possible, o.w. parse now
-  // ExtractPrepare() can only parse extract_field if user passes it as a string literal
-  if (tv.is_null) return BigIntVal::null();
-
-  TExtractField::type field;
-  void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
-  if (state != NULL) {
-    field = *reinterpret_cast<TExtractField::type*>(state);
-  } else {
-    field = StrToExtractField(context, unit_str);
-    if (field == TExtractField::INVALID_FIELD) {
-      string string_unit(reinterpret_cast<char*>(unit_str.ptr), unit_str.len);
-      context->SetError(Substitute("invalid extract field: $0", string_unit).c_str());
-      return BigIntVal::null();
-    }
-  }
-
-  const date& orig_date = *reinterpret_cast<const date*>(&tv.date);
-  const time_duration& time = *reinterpret_cast<const time_duration*>(&tv.time_of_day);
-
-  switch (field) {
-    case TExtractField::YEAR:
-    case TExtractField::QUARTER:
-    case TExtractField::MONTH:
-    case TExtractField::DAY:
-      if (orig_date.is_special()) return BigIntVal::null();
-      break;
-    case TExtractField::HOUR:
-    case TExtractField::MINUTE:
-    case TExtractField::SECOND:
-    case TExtractField::MILLISECOND:
-      if (time.is_special()) return BigIntVal::null();
-      break;
-    case TExtractField::EPOCH:
-      if (time.is_special() || orig_date.is_special()) return BigIntVal::null();
-      break;
-    case TExtractField::INVALID_FIELD:
-      DCHECK(false);
-  }
-
-  switch (field) {
-    case TExtractField::YEAR: {
-      return BigIntVal(orig_date.year());
-    }
-    case TExtractField::QUARTER: {
-      int m = orig_date.month();
-      return BigIntVal((m - 1) / 3 + 1);
-    }
-    case TExtractField::MONTH: {
-      return BigIntVal(orig_date.month());
-    }
-    case TExtractField::DAY: {
-      return BigIntVal(orig_date.day());
-    }
-    case TExtractField::HOUR: {
-      return BigIntVal(time.hours());
-    }
-    case TExtractField::MINUTE: {
-      return BigIntVal(time.minutes());
-    }
-    case TExtractField::SECOND: {
-      return BigIntVal(time.seconds());
-    }
-    case TExtractField::MILLISECOND: {
-      return BigIntVal(ExtractMillisecond(time));
-    }
-    case TExtractField::EPOCH: {
-      ptime epoch_date(date(1970, 1, 1), time_duration(0, 0, 0));
-      ptime cur_date(orig_date, time);
-      time_duration diff = cur_date - epoch_date;
-      return BigIntVal(diff.total_seconds());
-    }
-    default: {
-      DCHECK(false) << field;
-      return BigIntVal::null();
-    }
-  }
-}
-
-BigIntVal UdfBuiltins::Extract(FunctionContext* context, const TimestampVal& tv,
-    const StringVal& unit_str) {
-  return Extract(context, unit_str, tv);
-}
-
-void UdfBuiltins::ExtractPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope, int unit_idx) {
-  // Parse the unit up front if we can, otherwise do it on the fly in Extract()
-  if (ctx->IsArgConstant(unit_idx)) {
-    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(unit_idx));
-    TExtractField::type field = StrToExtractField(ctx, *unit_str);
-    if (field == TExtractField::INVALID_FIELD) {
-      string string_field(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute("invalid extract field: $0", string_field).c_str());
-    } else {
-      TExtractField::type* state = ctx->Allocate<TExtractField::type>();
-      RETURN_IF_NULL(ctx, state);
-      *state = field;
-      ctx->SetFunctionState(scope, state);
-    }
-  }
-}
-
-void UdfBuiltins::ExtractPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  ExtractPrepare(ctx, scope, 0);
-}
-
-void UdfBuiltins::SwappedExtractPrepare(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  ExtractPrepare(ctx, scope, 1);
-}
-
-void UdfBuiltins::ExtractClose(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  void* state = ctx->GetFunctionState(scope);
-  ctx->Free(reinterpret_cast<uint8_t*>(state));
-  ctx->SetFunctionState(scope, nullptr);
-}
-
 bool ValidateMADlibVector(FunctionContext* context, const StringVal& arr) {
   if (arr.ptr == NULL) {
     context->SetError("MADlib vector is null");
@@ -361,3 +198,135 @@ StringVal UdfBuiltins::DecodeVector(FunctionContext* context, const StringVal& a
   InplaceDoubleDecode(reinterpret_cast<char*>(result.ptr), arr.len);
   return result;
 }
+
+namespace {
+
+/// Used for closing TRUNC/DATE_TRUNC/EXTRACT/DATE_PART built-in functions.
+void CloseImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  void* state = ctx->GetFunctionState(scope);
+  ctx->Free(reinterpret_cast<uint8_t*>(state));
+  ctx->SetFunctionState(scope, nullptr);
+}
+
+}
+
+void UdfBuiltins::TruncForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return TruncForTimestampPrepareImpl(ctx, scope);
+}
+
+TimestampVal UdfBuiltins::TruncForTimestamp(FunctionContext* context,
+    const TimestampVal& tv, const StringVal &unit_str) {
+  return TruncForTimestampImpl(context, tv, unit_str);
+}
+
+void UdfBuiltins::TruncForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::TruncForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return TruncForDatePrepareImpl(ctx, scope);
+}
+
+DateVal UdfBuiltins::TruncForDate(FunctionContext* context, const DateVal& dv,
+    const StringVal &unit_str) {
+  return TruncForDateImpl(context, dv, unit_str);
+}
+
+void UdfBuiltins::TruncForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DateTruncForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return DateTruncForTimestampPrepareImpl(ctx, scope);
+}
+
+TimestampVal UdfBuiltins::DateTruncForTimestamp(FunctionContext* context,
+    const StringVal &unit_str, const TimestampVal& tv) {
+  return DateTruncForTimestampImpl(context, unit_str, tv);
+}
+
+void UdfBuiltins::DateTruncForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DateTruncForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return DateTruncForDatePrepareImpl(ctx, scope);
+}
+
+DateVal UdfBuiltins::DateTruncForDate(FunctionContext* context, const StringVal &unit_str,
+    const DateVal& dv) {
+  return DateTruncForDateImpl(context, unit_str, dv);
+}
+
+void UdfBuiltins::DateTruncForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::ExtractForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  ExtractForTimestampPrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::ExtractForTimestamp(FunctionContext* ctx, const TimestampVal& tv,
+    const StringVal& unit_str) {
+  return ExtractForTimestampImpl(ctx, tv, unit_str);
+}
+
+void UdfBuiltins::ExtractForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::ExtractForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  ExtractForDatePrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::ExtractForDate(FunctionContext* ctx, const DateVal& dv,
+    const StringVal& unit_str) {
+  return ExtractForDateImpl(ctx, dv, unit_str);
+}
+
+void UdfBuiltins::ExtractForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DatePartForTimestampPrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  DatePartForTimestampPrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::DatePartForTimestamp(FunctionContext* ctx,
+    const StringVal& unit_str, const TimestampVal& tv) {
+  return DatePartForTimestampImpl(ctx, unit_str, tv);
+}
+
+void UdfBuiltins::DatePartForTimestampClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
+
+void UdfBuiltins::DatePartForDatePrepare(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  DatePartForDatePrepareImpl(ctx, scope);
+}
+
+BigIntVal UdfBuiltins::DatePartForDate(FunctionContext* ctx, const StringVal& unit_str,
+    const DateVal& dv) {
+  return DatePartForDateImpl(ctx, unit_str, dv);
+}
+
+void UdfBuiltins::DatePartForDateClose(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return CloseImpl(ctx, scope);
+}
diff --git a/be/src/exprs/udf-builtins.cc b/be/src/exprs/udf-builtins.cc
index 61594ed..786e9cf 100644
--- a/be/src/exprs/udf-builtins.cc
+++ b/be/src/exprs/udf-builtins.cc
@@ -20,7 +20,10 @@
 
 #include "exprs/udf-builtins.h"
 
+#include <gutil/walltime.h>
+
 #include "gen-cpp/Exprs_types.h"
+#include "runtime/date-value.h"
 #include "runtime/runtime-state.h"
 #include "runtime/timestamp-value.h"
 #include "udf/udf-internal.h"
@@ -61,6 +64,7 @@ enum class TruncUnit {
 
 // Put non-exported functions in anonymous namespace to encourage inlining.
 namespace {
+
 // Returns the most recent date, no later than orig_date, which is on week_day
 // week_day: 0==Sunday, 1==Monday, ...
 date GoBackToWeekday(const date& orig_date, int week_day) {
@@ -245,7 +249,7 @@ TimestampValue TruncSecond(const date& orig_date, const time_duration& orig_time
 }
 
 // Truncate parts of milliseconds
-TimestampValue TruncMilliSeconds(const date& orig_date, const time_duration& orig_time) {
+TimestampValue TruncMilliseconds(const date& orig_date, const time_duration& orig_time) {
   time_duration new_time(orig_time.hours(), orig_time.minutes(), orig_time.seconds());
   // Fractional seconds are nanoseconds because Boost is configured to use nanoseconds
   // precision.
@@ -255,7 +259,7 @@ TimestampValue TruncMilliSeconds(const date& orig_date, const time_duration& ori
 }
 
 // Truncate parts of microseconds
-TimestampValue TruncMicroSeconds(const date& orig_date, const time_duration& orig_time) {
+TimestampValue TruncMicroseconds(const date& orig_date, const time_duration& orig_time) {
   time_duration new_time(orig_time.hours(), orig_time.minutes(), orig_time.seconds());
   // Fractional seconds are nanoseconds because Boost is configured to use nanoseconds
   // precision.
@@ -264,9 +268,9 @@ TimestampValue TruncMicroSeconds(const date& orig_date, const time_duration& ori
   return TimestampValue(orig_date, new_time);
 }
 
-// used by both Trunc and DateTrunc functions to perform the truncation
+// Used by both TRUNC and DATE_TRUNC functions to perform the truncation
 TimestampVal DoTrunc(
-    const TimestampValue ts, TruncUnit trunc_unit, FunctionContext* context) {
+    const TimestampValue& ts, TruncUnit trunc_unit, FunctionContext* ctx) {
   const date& orig_date = ts.date();
   const time_duration& orig_time = ts.time();
   TimestampValue ret;
@@ -354,115 +358,528 @@ TimestampVal DoTrunc(
       ret = TruncSecond(orig_date, orig_time);
       break;
     case TruncUnit::MILLISECONDS:
-      ret = TruncMilliSeconds(orig_date, orig_time);
+      ret = TruncMilliseconds(orig_date, orig_time);
       break;
     case TruncUnit::MICROSECONDS:
-      ret = TruncMicroSeconds(orig_date, orig_time);
+      ret = TruncMicroseconds(orig_date, orig_time);
       break;
     default:
       // internal error: implies StrToTruncUnit out of sync with this switch
-      context->SetError("truncate unit not supported");
+      ctx->SetError("truncate unit not supported");
       return TimestampVal::null();
   }
 
   ret.ToTimestampVal(&ret_val);
   return ret_val;
 }
-}
 
-TimestampVal UdfBuiltins::TruncImpl(
-    FunctionContext* context, const TimestampVal& tv, const StringVal& unit_str) {
-  if (tv.is_null) return TimestampVal::null();
-  TimestampValue ts = TimestampValue::FromTimestampVal(tv);
+// Returns the most recent date, no later than 'orig_date', which is on 'week_day'
+// 'week_day' is in [0, 6]; 0 = Monday, 6 = Sunday.
+DateValue GoBackToWeekday(const DateValue& orig_date, int week_day) {
+  DCHECK(orig_date.IsValid());
+  DCHECK(week_day >= 0 && week_day <= 6);
 
-  // resolve trunc_unit using the prepared state if possible, o.w. parse now
-  // TruncPrepare() can only parse trunc_unit if user passes it as a string literal
-  // TODO: it would be nice to resolve the branch before codegen so we can optimise
-  // this better.
-  TruncUnit trunc_unit;
-  void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
-  if (state != NULL) {
-    trunc_unit = *reinterpret_cast<TruncUnit*>(state);
+  // Week days are in [0, 6]; 0 = Monday, 6 = Sunday.
+  int current_week_day = orig_date.WeekDay();
+  DCHECK(current_week_day >= 0 && current_week_day <= 6);
+
+  if (current_week_day == week_day) {
+    return orig_date;
+  } else if (current_week_day > week_day) {
+    return orig_date.AddDays(week_day - current_week_day);
   } else {
-    trunc_unit = StrToTruncUnit(context, unit_str);
-    if (trunc_unit == TruncUnit::UNIT_INVALID) {
-      string string_unit(reinterpret_cast<char*>(unit_str.ptr), unit_str.len);
-      context->SetError(Substitute("Invalid Truncate Unit: $0", string_unit).c_str());
-      return TimestampVal::null();
+    return orig_date.AddDays(week_day - current_week_day - 7);
+  }
+}
+
+// Used by both TRUNC and DATE_TRUNC functions to perform the truncation
+DateVal DoTrunc(const DateValue& date, TruncUnit trunc_unit, FunctionContext* ctx) {
+  if (!date.IsValid()) return DateVal::null();
+
+  DCHECK(trunc_unit != TruncUnit::UNIT_INVALID
+      && trunc_unit != TruncUnit::MICROSECONDS
+      && trunc_unit != TruncUnit::HOUR
+      && trunc_unit != TruncUnit::MINUTE
+      && trunc_unit != TruncUnit::SECOND
+      && trunc_unit != TruncUnit::MILLISECONDS);
+
+  DateValue ret;
+
+  switch(trunc_unit) {
+    case TruncUnit::YEAR: {
+      int year;
+      discard_result(date.ToYear(&year));
+      ret = DateValue(year, 1, 1);
+      break;
+    }
+    case TruncUnit::QUARTER: {
+      int year, month, day;
+      discard_result(date.ToYearMonthDay(&year, &month, &day));
+      ret = DateValue(year, BitUtil::RoundDown(month - 1, 3) + 1, 1);
+      break;
+    }
+    case TruncUnit::MONTH: {
+      int year, month, day;
+      discard_result(date.ToYearMonthDay(&year, &month, &day));
+      ret = DateValue(year, month, 1);
+      break;
+    }
+    case TruncUnit::DAY: {
+      ret = date;
+      break;
+    }
+    case TruncUnit::WW: {
+      int year;
+      discard_result(date.ToYear(&year));
+      ret = GoBackToWeekday(date, DateValue(year, 1, 1).WeekDay());
+      break;
+    }
+    case TruncUnit::W: {
+      int year, month, day;
+      discard_result(date.ToYearMonthDay(&year, &month, &day));
+      ret = GoBackToWeekday(date, DateValue(year, month, 1).WeekDay());
+      break;
+    }
+    case TruncUnit::DAY_OF_WEEK: {
+      // Date of the previous Monday
+      ret = GoBackToWeekday(date, 0);
+      break;
+    }
+    case TruncUnit::WEEK: {
+      // ISO-8601 week starts on monday. go back to monday
+      ret = GoBackToWeekday(date, 0);
+      break;
+    }
+    case TruncUnit::MILLENNIUM: {
+      int year;
+      discard_result(date.ToYear(&year));
+      if (year <= 0) return DateVal::null();
+      // First year of current millennium is 2001
+      ret = DateValue((year - 1) / 1000 * 1000 + 1, 1, 1);
+      break;
+    }
+    case TruncUnit::CENTURY: {
+      int year;
+      discard_result(date.ToYear(&year));
+      if (year <= 0) return DateVal::null();
+      // First year of current century is 2001
+      ret = DateValue((year - 1) / 100 * 100 + 1, 1, 1);
+      break;
+    }
+    case TruncUnit::DECADE: {
+      int year;
+      // Decades start with years ending in '0'.
+      discard_result(date.ToYear(&year));
+      ret = DateValue(year / 10 * 10, 1, 1);
+      break;
     }
+    default:
+      // internal error: implies StrToTruncUnit out of sync with this switch
+      ctx->SetError("truncate unit not supported");
+      return DateVal::null();
   }
-  return DoTrunc(ts, trunc_unit, context);
+
+  return ret.ToDateVal();
 }
 
-void UdfBuiltins::TruncPrepare(
-    FunctionContext* ctx, FunctionContext::FunctionStateScope scope) {
-  // Parse the unit up front if we can, otherwise do it on the fly in Trunc()
-  if (ctx->IsArgConstant(1)) {
-    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(1));
-    TruncUnit trunc_unit = StrToTruncUnit(ctx, *unit_str);
-    if (trunc_unit == TruncUnit::UNIT_INVALID) {
-      string string_unit(reinterpret_cast<char*>(unit_str->ptr), unit_str->len);
-      ctx->SetError(Substitute("Invalid Truncate Unit: $0", string_unit).c_str());
-    } else {
-      TruncUnit* state = ctx->Allocate<TruncUnit>();
-      RETURN_IF_NULL(ctx, state);
-      *state = trunc_unit;
-      ctx->SetFunctionState(scope, state);
+// Maps the user facing name of a unit to a TExtractField
+// Returns the TExtractField for the given unit
+TExtractField::type StrToExtractField(FunctionContext* ctx,
+    const StringVal& unit_str) {
+  StringVal unit = UdfBuiltins::Lower(ctx, unit_str);
+  if (UNLIKELY(unit.is_null)) return TExtractField::INVALID_FIELD;
+  if (unit == "year") return TExtractField::YEAR;
+  if (unit == "quarter") return TExtractField::QUARTER;
+  if (unit == "month") return TExtractField::MONTH;
+  if (unit == "day") return TExtractField::DAY;
+  if (unit == "hour") return TExtractField::HOUR;
+  if (unit == "minute") return TExtractField::MINUTE;
+  if (unit == "second") return TExtractField::SECOND;
+  if (unit == "millisecond") return TExtractField::MILLISECOND;
+  if (unit == "epoch") return TExtractField::EPOCH;
+  return TExtractField::INVALID_FIELD;
+}
+
+static int64_t ExtractMillisecond(const time_duration& time) {
+  // Fractional seconds are nanoseconds because Boost is configured
+  // to use nanoseconds precision
+  return time.fractional_seconds() / (NANOS_PER_MICRO * MICROS_PER_MILLI)
+       + time.seconds() * MILLIS_PER_SEC;
+}
+
+// Used by both EXTRACT and DATE_PART functions to perform field extraction.
+BigIntVal DoExtract(const TimestampValue& tv, TExtractField::type field,
+    FunctionContext* ctx) {
+  switch (field) {
+    case TExtractField::YEAR:
+    case TExtractField::QUARTER:
+    case TExtractField::MONTH:
+    case TExtractField::DAY:
+      if (!tv.HasDate()) return BigIntVal::null();
+      break;
+    case TExtractField::HOUR:
+    case TExtractField::MINUTE:
+    case TExtractField::SECOND:
+    case TExtractField::MILLISECOND:
+      if (!tv.HasTime()) return BigIntVal::null();
+      break;
+    case TExtractField::EPOCH:
+      if (!tv.HasDateAndTime()) return BigIntVal::null();
+      break;
+    case TExtractField::INVALID_FIELD:
+      DCHECK(false);
+  }
+
+  const date& orig_date = tv.date();
+  const time_duration& time = tv.time();
+
+  switch (field) {
+    case TExtractField::YEAR: {
+      return BigIntVal(orig_date.year());
+    }
+    case TExtractField::QUARTER: {
+      int m = orig_date.month();
+      return BigIntVal((m - 1) / 3 + 1);
+    }
+    case TExtractField::MONTH: {
+      return BigIntVal(orig_date.month());
+    }
+    case TExtractField::DAY: {
+      return BigIntVal(orig_date.day());
+    }
+    case TExtractField::HOUR: {
+      return BigIntVal(time.hours());
+    }
+    case TExtractField::MINUTE: {
+      return BigIntVal(time.minutes());
+    }
+    case TExtractField::SECOND: {
+      return BigIntVal(time.seconds());
+    }
+    case TExtractField::MILLISECOND: {
+      return BigIntVal(ExtractMillisecond(time));
+    }
+    case TExtractField::EPOCH: {
+      ptime epoch_date(date(1970, 1, 1), time_duration(0, 0, 0));
+      ptime cur_date(orig_date, time);
+      time_duration diff = cur_date - epoch_date;
+      return BigIntVal(diff.total_seconds());
+    }
+    default: {
+      // internal error: implies StrToExtractField out of sync with this switch
+      ctx->SetError("extract unit not supported");
+      return BigIntVal::null();
     }
   }
 }
 
-void UdfBuiltins::TruncClose(FunctionContext* ctx,
-    FunctionContext::FunctionStateScope scope) {
-  void* state = ctx->GetFunctionState(scope);
-  ctx->Free(reinterpret_cast<uint8_t*>(state));
-  ctx->SetFunctionState(scope, nullptr);
+// Used by both EXTRACT and DATE_PART functions to perform field extraction.
+BigIntVal DoExtract(const DateValue& dv, TExtractField::type field,
+    FunctionContext* ctx) {
+  if (!dv.IsValid()) return BigIntVal::null();
+
+  DCHECK(field != TExtractField::INVALID_FIELD
+      && field != TExtractField::HOUR
+      && field != TExtractField::MINUTE
+      && field != TExtractField::SECOND
+      && field != TExtractField::MILLISECOND
+      && field != TExtractField::EPOCH);
+
+  switch (field) {
+    case TExtractField::YEAR: {
+      int year;
+      discard_result(dv.ToYear(&year));
+      return BigIntVal(year);
+    }
+    case TExtractField::QUARTER: {
+      int year, month, day;
+      discard_result(dv.ToYearMonthDay(&year, &month, &day));
+      return BigIntVal((month - 1) / 3 + 1);
+    }
+    case TExtractField::MONTH: {
+      int year, month, day;
+      discard_result(dv.ToYearMonthDay(&year, &month, &day));
+      return BigIntVal(month);
+    }
+    case TExtractField::DAY: {
+      int year, month, day;
+      discard_result(dv.ToYearMonthDay(&year, &month, &day));
+      return BigIntVal(day);
+    }
+    default: {
+      // internal error: implies StrToExtractField out of sync with this switch
+      ctx->SetError("extract unit not supported");
+      return BigIntVal::null();
+    }
+  }
+}
+
+inline TimestampValue FromVal(const TimestampVal& val) {
+  return TimestampValue::FromTimestampVal(val);
+}
+
+inline DateValue FromVal(const DateVal& val) {
+  return DateValue::FromDateVal(val);
+}
+
+inline bool IsTimeOfDayUnit(TruncUnit unit) {
+  return (unit == TruncUnit::HOUR
+      || unit == TruncUnit::MINUTE
+      || unit == TruncUnit::SECOND
+      || unit == TruncUnit::MILLISECONDS
+      || unit == TruncUnit::MICROSECONDS);
+}
+
+inline bool IsTimeOfDayUnit(TExtractField::type unit) {
+  return (unit == TExtractField::HOUR
+      || unit == TExtractField::MINUTE
+      || unit == TExtractField::SECOND
+      || unit == TExtractField::MILLISECOND
+      || unit == TExtractField::EPOCH);
 }
 
-TimestampVal UdfBuiltins::DateTruncImpl(
-    FunctionContext* context, const TimestampVal& tv, const StringVal& unit_str) {
-  if (tv.is_null) return TimestampVal::null();
-  TimestampValue ts = TimestampValue::FromTimestampVal(tv);
+inline bool IsInvalidUnit(TruncUnit unit) {
+  return (unit == TruncUnit::UNIT_INVALID);
+}
+
+inline bool IsInvalidUnit(TExtractField::type unit) {
+  return (unit == TExtractField::INVALID_FIELD);
+}
 
-  // resolve date_trunc_unit using the prepared state if possible, o.w. parse now
-  // DateTruncPrepare() can only parse trunc_unit if user passes it as a string literal
-  TruncUnit date_trunc_unit;
-  void* state = context->GetFunctionState(FunctionContext::THREAD_LOCAL);
+/// Used for TRUNC/DATE_TRUNC/EXTRACT/DATE_PART built-in functions.
+/// ALLOW_TIME_OF_DAY_UNIT: true iff the built-in function call accepts time-of-day units.
+/// UdfType: udf type the built-in function works with.
+/// InternalType: Impla's internal type that corresponds to UdfType.
+/// ReturnUdfType: The built-in function's return type.
+/// UnitType: type to represent unit values.
+/// to_unit: function to parse unit strings.
+/// do_func: function to implement the built-in function.
+/// func_descr: description of the built-in function.
+/// unit_descr: description of the unit parameter.
+template <
+    bool ALLOW_TIME_OF_DAY_UNIT,
+    typename UdfType,
+    typename InternalType,
+    typename ReturnUdfType,
+    typename UnitType,
+    UnitType to_unit(FunctionContext*, const StringVal&),
+    ReturnUdfType do_func(const InternalType&, UnitType, FunctionContext*)>
+ReturnUdfType ExtractTruncFuncTempl(FunctionContext* ctx, const UdfType& val,
+    const StringVal& unit_str, const string& func_descr, const string& unit_descr) {
+  if (val.is_null) return ReturnUdfType::null();
+
+  // resolve 'unit' using the prepared state if possible, o.w. parse now
+  // ExtractTruncFuncPrepareTempl() can only parse unit if user passes it as a string
+  // literal
+  // TODO: it would be nice to resolve the branch before codegen so we can optimise
+  // this better.
+  UnitType unit;
+  void* state = ctx->GetFunctionState(FunctionContext::THREAD_LOCAL);
   if (state != NULL) {
-    date_trunc_unit = *reinterpret_cast<TruncUnit*>(state);
+    unit = *reinterpret_cast<UnitType*>(state);
   } else {
-    date_trunc_unit = StrToDateTruncUnit(context, unit_str);
-    if (date_trunc_unit == TruncUnit::UNIT_INVALID) {
+    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);
-      context->SetError(
-          Substitute("Invalid Date Truncate Unit: $0", string_unit).c_str());
-      return TimestampVal::null();
+      ctx->SetError(Substitute(
+          "Unsupported $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
+      return ReturnUdfType::null();
+    } 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());
+      return ReturnUdfType::null();
     }
   }
-  return DoTrunc(ts, date_trunc_unit, context);
+  return do_func(FromVal(val), unit, ctx);
 }
 
-void UdfBuiltins::DateTruncPrepare(
-    FunctionContext* ctx, FunctionContext::FunctionStateScope scope) {
-  // Parse the unit up front if we can, otherwise do it on the fly in DateTrunc()
-  if (ctx->IsArgConstant(0)) {
-    StringVal* unit_str = reinterpret_cast<StringVal*>(ctx->GetConstantArg(0));
-    TruncUnit date_trunc_unit = StrToDateTruncUnit(ctx, *unit_str);
-    if (date_trunc_unit == TruncUnit::UNIT_INVALID) {
+/// Does the preparation for TRUNC/DATE_TRUNC/EXTRACT/DATE_PART built-in functions.
+/// ALLOW_TIME_OF_DAY_UNIT: true iff the built-in function call accepts time-of-day units.
+/// UNIT_IDX: indicates which parameter of the function call is the unit parameter.
+/// UnitType: type to represent unit values.
+/// to_unit: function to parse unit strings.
+/// func_descr: description of the built-in function.
+/// unit_descr: description of the unit parameter.
+template <
+    bool ALLOW_TIME_OF_DAY_UNIT,
+    int UNIT_IDX,
+    typename UnitType,
+    UnitType to_unit(FunctionContext*, const StringVal&)>
+void ExtractTruncFuncPrepareTempl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope,
+    const string& func_descr, const string& unit_descr) {
+  // 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 Date Truncate Unit: $0", string_unit).c_str());
+      ctx->SetError(Substitute(
+          "Invalid $0 $1: $2", func_descr, unit_descr, string_unit).c_str());
     } else {
-      TruncUnit* state = ctx->Allocate<TruncUnit>();
+      UnitType* state = ctx->Allocate<UnitType>();
       RETURN_IF_NULL(ctx, state);
-      *state = date_trunc_unit;
+      *state = unit;
       ctx->SetFunctionState(scope, state);
     }
   }
 }
 
-void UdfBuiltins::DateTruncClose(
-    FunctionContext* ctx, FunctionContext::FunctionStateScope scope) {
-  void* state = ctx->GetFunctionState(scope);
-  ctx->Free(reinterpret_cast<uint8_t*>(state));
-  ctx->SetFunctionState(scope, nullptr);
+}
+
+void UdfBuiltins::TruncForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      1,
+      TruncUnit,
+      StrToTruncUnit>(ctx, scope, "Truncate", "Unit");
+}
+
+TimestampVal UdfBuiltins::TruncForTimestampImpl(FunctionContext* ctx,
+    const TimestampVal& tv, const StringVal &unit_str) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      TimestampVal,
+      TruncUnit,
+      StrToTruncUnit,
+      DoTrunc>(ctx, tv, unit_str, "Truncate", "Unit");
+}
+
+void UdfBuiltins::TruncForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      1,
+      TruncUnit,
+      StrToTruncUnit>(ctx, scope, "Truncate", "Unit");
+}
+
+DateVal UdfBuiltins::TruncForDateImpl(FunctionContext* ctx, const DateVal& dv,
+    const StringVal &unit_str) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      DateVal,
+      TruncUnit,
+      StrToTruncUnit,
+      DoTrunc>(ctx, dv, unit_str, "Truncate", "Unit");
+}
+
+void UdfBuiltins::DateTruncForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      0,
+      TruncUnit,
+      StrToDateTruncUnit>(ctx, scope, "Date Truncate", "Unit");
+}
+
+TimestampVal UdfBuiltins::DateTruncForTimestampImpl(FunctionContext* ctx,
+    const StringVal &unit_str, const TimestampVal& tv) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      TimestampVal,
+      TruncUnit,
+      StrToDateTruncUnit,
+      DoTrunc>(ctx, tv, unit_str, "Date Truncate", "Unit");
+}
+
+void UdfBuiltins::DateTruncForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      0,
+      TruncUnit,
+      StrToDateTruncUnit>(ctx, scope, "Date Truncate", "Unit");
+}
+
+DateVal UdfBuiltins::DateTruncForDateImpl(FunctionContext* ctx, const StringVal &unit_str,
+    const DateVal& dv) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      DateVal,
+      TruncUnit,
+      StrToDateTruncUnit,
+      DoTrunc>(ctx, dv, unit_str, "Date Truncate", "Unit");
+}
+
+void UdfBuiltins::ExtractForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      1,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Extract", "Field");
+}
+
+BigIntVal UdfBuiltins::ExtractForTimestampImpl(FunctionContext* ctx,
+    const TimestampVal& tv, const StringVal& unit_str) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, tv, unit_str, "Extract", "Field");
+}
+
+void UdfBuiltins::ExtractForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      1,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Extract", "Field");
+}
+
+BigIntVal UdfBuiltins::ExtractForDateImpl(FunctionContext* ctx, const DateVal& dv,
+    const StringVal& unit_str) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, dv, unit_str, "Extract", "Field");
+}
+
+void UdfBuiltins::DatePartForTimestampPrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<true,
+      0,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Date Part", "Field");
+}
+
+BigIntVal UdfBuiltins::DatePartForTimestampImpl(FunctionContext* ctx,
+    const StringVal& unit_str, const TimestampVal& tv) {
+  return ExtractTruncFuncTempl<true,
+      TimestampVal,
+      TimestampValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, tv, unit_str, "Date Part", "Field");
+}
+
+void UdfBuiltins::DatePartForDatePrepareImpl(FunctionContext* ctx,
+    FunctionContext::FunctionStateScope scope) {
+  return ExtractTruncFuncPrepareTempl<false,
+      0,
+      TExtractField::type,
+      StrToExtractField>(ctx, scope, "Date Part", "Field");
+}
+
+BigIntVal UdfBuiltins::DatePartForDateImpl(FunctionContext* ctx,
+    const StringVal& unit_str, const DateVal& dv) {
+  return ExtractTruncFuncTempl<false,
+      DateVal,
+      DateValue,
+      BigIntVal,
+      TExtractField::type,
+      StrToExtractField,
+      DoExtract>(ctx, dv, unit_str, "Date Part", "Field");
 }
diff --git a/be/src/exprs/udf-builtins.h b/be/src/exprs/udf-builtins.h
index 8aec873..430a0fa 100644
--- a/be/src/exprs/udf-builtins.h
+++ b/be/src/exprs/udf-builtins.h
@@ -33,6 +33,7 @@ using impala_udf::DoubleVal;
 using impala_udf::TimestampVal;
 using impala_udf::StringVal;
 using impala_udf::DecimalVal;
+using impala_udf::DateVal;
 
 /// Builtins written against the UDF interface. The builtins in the other files
 /// should be replaced to the UDF interface as well.
@@ -56,6 +57,7 @@ class UdfBuiltins {
   static BooleanVal IsNan(FunctionContext* context, const DoubleVal& val);
   static BooleanVal IsInf(FunctionContext* context, const DoubleVal& val);
 
+  /// This is for TRUNC(TIMESTAMP, STRING) function.
   /// Rounds (truncating down) a Timestamp to the specified unit.
   ///    Units:
   ///    CC, SCC : One greater than the first two digits of
@@ -63,25 +65,36 @@ class UdfBuiltins {
   ///    SYYYY, YYYY, YEAR, SYEAR, YYY, YY, Y : Current Year
   ///    Q : Quarter
   ///    MONTH, MON, MM, RM : Month
-  ///    WW : Same day of the week as the first day of the year
-  ///    W : Same day of the week as the first day of the month
   ///    DDD, DD, J : Day
   ///    DAY, DY, D : Starting day of the week
+  ///    WW : Truncates to the most recent date, no later than 'tv', which is on the same
+  ///         day of the week as the first day of year.
+  ///    W : Truncates to the most recent date, no later than 'tv', which is on the same
+  ///        day of the week as the first day of month.
   ///    HH, HH12, HH24 : Hour
   ///    MI : Minute
   ///
   ///    Reference:
   ///    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions201.htm
-  static TimestampVal Trunc(FunctionContext* context, const TimestampVal& date,
-      const StringVal& unit_str);
-  /// Implementation of Trunc, not cross-compiled.
-  static TimestampVal TruncImpl(FunctionContext* context, const TimestampVal& date,
-      const StringVal& unit_str);
-  static void TruncPrepare(FunctionContext* context,
+  static TimestampVal TruncForTimestamp(FunctionContext* ctx, const TimestampVal& tv,
+      const StringVal &unit_str);
+  static void TruncForTimestampPrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void TruncForTimestampClose(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
-  static void TruncClose(
-      FunctionContext* context, FunctionContext::FunctionStateScope scope);
 
+  /// This for TRUNC(DATE, STRING) function.
+  /// Rounds (truncating down) a Date to the specified unit.
+  /// Works as 'TruncForTimestamp' but doesn't accept time of day units: HH, HH12, HH24,
+  /// MI.
+  static DateVal TruncForDate(FunctionContext* ctx, const DateVal& dv,
+      const StringVal &unit_str);
+  static void TruncForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void TruncForDateClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+
+  /// This for DATE_TRUNC(STRING, TIMESTAMP) function.
   /// Rounds (truncating down) a Timestamp to the specified unit.
   ///    Units:
   ///    MILLENNIUM: The millennium number.
@@ -96,44 +109,69 @@ class UdfBuiltins {
   ///    SECOND: The seconds field (0–59).
   ///    MILLISECONDS: The milliseconds fraction in the seconds.
   ///    MICROSECONDS: The microseconds fraction in the seconds.
-
+  ///
   ///    Reference:
   ///    https://my.vertica.com/docs/8.1.x/HTML/index.htm#Authoring/
   ///       SQLReferenceManual/Functions/Date-Time/DATE_TRUNC.htm
-  static TimestampVal DateTrunc(
-      FunctionContext* context, const StringVal& unit_str, const TimestampVal& date);
-  /// Implementation of DateTrunc, not cross-compiled.
-  static TimestampVal DateTruncImpl(
-      FunctionContext* context, const TimestampVal& date, const StringVal& unit_str);
-  static void DateTruncPrepare(
-      FunctionContext* context, FunctionContext::FunctionStateScope scope);
-  static void DateTruncClose(
-      FunctionContext* context, FunctionContext::FunctionStateScope scope);
+  static TimestampVal DateTruncForTimestamp(FunctionContext* ctx,
+      const StringVal &unit_str, const TimestampVal& tv);
+  static void DateTruncForTimestampPrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void DateTruncForTimestampClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+
+  /// This is for DATE_TRUNC(STRING, DATE) function.
+  /// Rounds (truncating down) a Date to the specified unit.
+  /// Works as 'DateTruncForTimestamp' but doesn't accept time of day units: HOUR, MINUTE,
+  /// SECOND, MILLISECONDS, MICROSECONDS.
+  static DateVal DateTruncForDate(FunctionContext* ctx, const StringVal &unit_str,
+      const DateVal& dv);
+  static void DateTruncForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static void DateTruncForDateClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
 
+  /// This is for the EXTRACT(TIMESTAMP, STRING) and EXTRACT(TIMEUNIT FROM TIMESTAMP)
+  /// functions.
   /// Returns a single field from a timestamp
   ///    Fields:
   ///      YEAR, MONTH, DAY, HOUR, MINUTE, SECOND, MILLISECOND, EPOCH
-  ///    Reference:
-  ///    http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
-  ///
-  /// This is used by the DATE_PART function.
-  static BigIntVal Extract(FunctionContext* context, const StringVal& field_str,
-      const TimestampVal& date);
+  /// Reference:
+  /// http://docs.oracle.com/cd/B19306_01/server.102/b14200/functions050.htm
+  static void ExtractForTimestampPrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForTimestamp(FunctionContext* ctx, const TimestampVal& tv,
+      const StringVal& unit_str);
+  static void ExtractForTimestampClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
 
-  /// This is for the EXTRACT(Timestamp, String) and EXTRACT(Timeunit FROM
-  /// Timestamp) functions.
-  static BigIntVal Extract(FunctionContext* context, const TimestampVal& date,
-      const StringVal& field_str);
-  /// This is used by the DATE_PART function.
-  static void ExtractPrepare(FunctionContext* context,
+  /// This is for the EXTRACT(DATE, STRING) and EXTRACT(TIMEUNIT FROM DATE)
+  /// functions.
+  /// Works as 'ExtractForTimestamp' but doesn't accept time of day fields: HOUR, MINUTE,
+  /// SECOND, MILLISECOND, EPOCH.
+  static void ExtractForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForDate(FunctionContext* ctx, const DateVal& dv,
+      const StringVal& unit_str);
+  static void ExtractForDateClose(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
 
-  /// This is for the EXTRACT(Timestamp, String) and EXTRACT(Timeunit FROM
-  /// Timestamp) functions.
-  static void SwappedExtractPrepare(FunctionContext* context,
+  /// This is for DATE_PART(STRING, TIMESTAMP) function.
+  /// Similar to 'ExtractForTimestamp' with the argument order reversed.
+  static void DatePartForTimestampPrepare(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
-  /// This is used by both EXTRACT and DATE_PART
-  static void ExtractClose(FunctionContext* context,
+  static BigIntVal DatePartForTimestamp(FunctionContext* ctx, const StringVal& unit_str,
+      const TimestampVal& tv);
+  static void DatePartForTimestampClose(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+
+  /// This is for DATE_PART(STRING, DATE) function.
+  /// Similar to 'ExtractForDate' with the argument order reversed.
+  static void DatePartForDatePrepare(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal DatePartForDate(FunctionContext* ctx, const StringVal& unit_str,
+      const DateVal& dv);
+  static void DatePartForDateClose(FunctionContext* ctx,
       FunctionContext::FunctionStateScope scope);
 
   /// Converts a set of doubles to double[] stored as a StringVal
@@ -159,11 +197,49 @@ class UdfBuiltins {
   static StringVal DecodeVector(FunctionContext* context, const StringVal& arr);
 
  private:
-  /// Does the preparation for EXTRACT. The unit_idx parameter should indicate which
-  /// parameter of the EXTRACT call is the time unit param. DATE_PART will also use this
-  /// with a different unit_idx than EXTRACT.
-  static void ExtractPrepare(FunctionContext* context,
-      FunctionContext::FunctionStateScope scope, int unit_idx);
+  /// Implementation of TruncForTimestamp, not cross-compiled.
+  static void TruncForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static TimestampVal TruncForTimestampImpl(FunctionContext* context,
+      const TimestampVal& tv, const StringVal &unit_str);
+  /// Implementation of TruncForDate, not cross-compiled.
+  static void TruncForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static DateVal TruncForDateImpl(FunctionContext* context, const DateVal& dv,
+      const StringVal &unit_str);
+
+  /// Implementation of DateTruncForTimestamp, not cross-compiled.
+  static void DateTruncForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static TimestampVal DateTruncForTimestampImpl(FunctionContext* context,
+      const StringVal &unit_str, const TimestampVal& tv);
+  /// Implementation of DateTruncForDate, not cross-compiled.
+  static void DateTruncForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static DateVal DateTruncForDateImpl(FunctionContext* context, const StringVal &unit_str,
+      const DateVal& dv);
+
+  /// Implementation of ExtractForTimestamp, not cross-compiled.
+  static void ExtractForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForTimestampImpl(FunctionContext* context,
+      const TimestampVal& tv, const StringVal& unit_str);
+  /// Implementation of ExtractForDate, not cross-compiled.
+  static void ExtractForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal ExtractForDateImpl(FunctionContext* context, const DateVal& dv,
+      const StringVal& unit_st);
+
+  /// Implementation of DatePartForTimestamp, not cross-compiled.
+  static void DatePartForTimestampPrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal DatePartForTimestampImpl(FunctionContext* context,
+      const StringVal& unit_str, const TimestampVal& tv);
+  /// Implementation of DatePartForDate, not cross-compiled.
+  static void DatePartForDatePrepareImpl(FunctionContext* ctx,
+      FunctionContext::FunctionStateScope scope);
+  static BigIntVal DatePartForDateImpl(FunctionContext* context,
+      const StringVal& unit_str, const DateVal& dv);
 };
 
 } // namespace impala
diff --git a/be/src/runtime/date-test.cc b/be/src/runtime/date-test.cc
index 95f90e3..0b8f804 100644
--- a/be/src/runtime/date-test.cc
+++ b/be/src/runtime/date-test.cc
@@ -593,6 +593,93 @@ TEST(DateTest, DateValueEdgeCases) {
   }
 }
 
+TEST(DateTest, AddDays) {
+  // Adding days to an invalid DateValue instance returns an invalid DateValue.
+  DateValue invalid_dv;
+  EXPECT_FALSE(invalid_dv.IsValid());
+  EXPECT_FALSE(invalid_dv.AddDays(1).IsValid());
+
+  // AddDays works with 0, > 0 and < 0 number of days.
+  DateValue dv(2019, 5, 16);
+  EXPECT_EQ(DateValue(2019, 5, 17), dv.AddDays(1));
+  EXPECT_EQ(DateValue(2019, 5, 15), dv.AddDays(-1));
+  // May has 31 days, April has 30 days.
+  EXPECT_EQ(DateValue(2019, 6, 16), dv.AddDays(31));
+  EXPECT_EQ(DateValue(2019, 4, 16), dv.AddDays(-30));
+  // 2019 is not a leap year, 2020 is a leap year.
+  EXPECT_EQ(DateValue(2020, 5, 16), dv.AddDays(366));
+  EXPECT_EQ(DateValue(2018, 5, 16), dv.AddDays(-365));
+
+  // 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
+  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
+  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
+  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, 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.
+  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.
+  EXPECT_EQ(4, DateValue(9999, 12, 31).WeekDay());
+
+  // Test lower limit.
+  // 0000.01.01 is Monday.
+  EXPECT_EQ(0, DateValue(1, 1, 1).WeekDay());
+  // 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.
+  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());
+  do {
+    int y, m, d;
+    EXPECT_TRUE(dv.ToYearMonthDay(&y, &m, &d));
+
+    EXPECT_TRUE(dv.ToYear(&year));
+    EXPECT_EQ(y, year);
+
+    dv = dv.AddDays(1);
+  } while (dv.IsValid());
+}
+
 }
 
 IMPALA_TEST_MAIN();
diff --git a/be/src/runtime/date-value.cc b/be/src/runtime/date-value.cc
index a9dd44e..54b31da 100644
--- a/be/src/runtime/date-value.cc
+++ b/be/src/runtime/date-value.cc
@@ -28,7 +28,8 @@
 namespace impala {
 
 namespace {
-const cctz::civil_day EPOCH_DATE(1970, 1, 1);
+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;
@@ -97,6 +98,97 @@ bool DateValue::ToYearMonthDay(int* year, int* month, int* day) const {
   return true;
 }
 
+namespace {
+
+inline int32_t CalcFirstDayOfYearSinceEpoch(int year) {
+  int m400 = year % 400;
+  int m100 = m400 % 100;
+  int m4 = m100 % 4;
+
+  return (year - EPOCH_YEAR) * 365
+      + ((year - EPOCH_YEAR / 4 * 4 + ((m4 != 0) ? 4 - m4 : 0)) / 4 - 1)
+      - ((year - EPOCH_YEAR / 100 * 100 + ((m100 != 0) ? 100 - m100 : 0)) / 100 - 1)
+      + ((year - EPOCH_YEAR / 400 * 400 + ((m400 != 0) ? 400 - m400 : 0)) / 400 - 1);
+}
+
+}
+
+bool DateValue::ToYear(int* year) const {
+  DCHECK(year != nullptr);
+  if (UNLIKELY(!IsValid())) return false;
+
+  // This function was introduced to extract year of a DateValue efficiently.
+  // It will be fast for most days of the year and only slightly slower for days around
+  // the beginning and end of the year.
+  //
+  // Here's a quick explanation. Let's use the following notation:
+  // m400 = year % 400
+  // m100 = m400 % 100
+  // m4 = m100 % 4
+  //
+  // If 'days' is the number of days between 1970-01-01 and the first day of 'year'
+  // (excluding the endpoint), then the following is true:
+  // days == (year - 1970) * 365
+  //       + ((year - 1968 + ((m4 != 0) ? 4 - m4 : 0)) / 4 - 1)
+  //       - ((year - 1900 + ((m100 != 0) ? 100 - m100 : 0)) / 100 - 1)
+  //       + ((year - 1600 + ((m400 != 0) ? 400 - m400 : 0)) / 400 - 1)
+  //
+  // Reordering the equation we get:
+  // days * 400 == (year - 1970) * 365 * 400
+  //       + ((year - 1968) * 100 + ((m4 != 0) ? (4 - m4) * 100 : 0) - 400)
+  //       - ((year - 1900) * 4 + ((m100 != 0) ? (100 - m100) * 4 : 0) - 400)
+  //       + (year - 1600 + ((m400 != 0) ? 400 - m400 : 0) - 400)
+  //
+  // then:
+  // days * 400 == year * 146000 - 287620000
+  //       + (year * 100 - 196800 + ((m4 != 0) ? (4 - m4) * 100 : 0) - 400)
+  //       - (year * 4 - 7600 + ((m100 != 0) ? (100 - m100) * 4 : 0) - 400)
+  //       + (year - 1600 + ((m400 != 0) ? 400 - m400 : 0) - 400)
+  //
+  // which means that (A):
+  // year * 146097 == days * 400 + 287811200
+  //       - ((m4 != 0) ? (4 - m4) * 100 : 0)
+  //       + ((m100 != 0) ? (100 - m100) * 4 : 0)
+  //       - ((m400 != 0) ? 400 - m400 : 0)
+  //
+  // On the other hand, if
+  // f(year) = - ((m4 != 0) ? (4 - m4) * 100 : 0)
+  //           + ((m100 != 0) ? (100 - m100) * 4 : 0)
+  //           - ((m400 != 0) ? 400 - m400 : 0)
+  // and 'year' is in the [0, 9999] range, then it follows that (B):
+  // f(year) must fall into the [-591, 288] range.
+  //
+  // Finally, if we put (A) and (B) together we can conclude that 'year' must fall into
+  // the
+  // [ (days * 400 + 287811200 - 591) / 146097, (days * 400 + 287811200 + 288) / 146097 ]
+  // range.
+
+  int tmp = days_since_epoch_ * 400 + 287811200;
+  int first_year = (tmp - 591) / 146097;
+  int last_year = (tmp + 288) / 146097;
+
+  if (first_year == last_year) {
+    *year = first_year;
+  } else if (CalcFirstDayOfYearSinceEpoch(last_year) <= days_since_epoch_) {
+    *year = last_year;
+  } else {
+    *year = first_year;
+  }
+
+  return true;
+}
+
+int DateValue::WeekDay() const {
+  if (UNLIKELY(!IsValid())) return -1;
+  const cctz::civil_day cd = EPOCH_DATE + days_since_epoch_;
+  return static_cast<int>(cctz::get_weekday(cd));
+}
+
+DateValue DateValue::AddDays(int days) const {
+  if (UNLIKELY(!IsValid())) return DateValue();
+  return DateValue(days_since_epoch_ + days);
+}
+
 bool DateValue::ToDaysSinceEpoch(int32_t* days) const {
   DCHECK(days != nullptr);
   if (UNLIKELY(!IsValid())) return false;
diff --git a/be/src/runtime/date-value.h b/be/src/runtime/date-value.h
index e927d7d..606eb7a 100644
--- a/be/src/runtime/date-value.h
+++ b/be/src/runtime/date-value.h
@@ -82,6 +82,20 @@ class DateValue {
   /// Otherwise, return false.
   bool ToYearMonthDay(int* year, int* month, int* day) const WARN_UNUSED_RESULT;
 
+  /// If this DateValue instance is valid, convert it to year and return true. Result is
+  /// placed in 'year'.
+  /// 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
+  /// 6 = Sunday.
+  /// Otherwise, return -1.
+  int WeekDay() const;
+
+  /// If this DateValue instance valid, add 'days' to it and return the result.
+  /// Otherwise, return an invalid DateValue instance.
+  DateValue AddDays(int days) 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.
diff --git a/common/function-registry/impala_functions.py b/common/function-registry/impala_functions.py
index e702db7..44ee32e 100644
--- a/common/function-registry/impala_functions.py
+++ b/common/function-registry/impala_functions.py
@@ -81,21 +81,33 @@ visible_functions = [
   [['is_inf'], 'BOOLEAN', ['DOUBLE'],
    '_ZN6impala11UdfBuiltins5IsInfEPN10impala_udf15FunctionContextERKNS1_9DoubleValE'],
   [['trunc'], 'TIMESTAMP', ['TIMESTAMP', 'STRING'],
-   '_ZN6impala11UdfBuiltins5TruncEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
-   '_ZN6impala11UdfBuiltins12TruncPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins10TruncCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins17TruncForTimestampEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins24TruncForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins22TruncForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['trunc'], 'DATE', ['DATE', 'STRING'],
+   '_ZN6impala11UdfBuiltins12TruncForDateEPN10impala_udf15FunctionContextERKNS1_7DateValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins19TruncForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins17TruncForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
   # Don't add an entry for EXTRACT(STRING, TIMESTAMP). STRINGs may be used to represent
   # TIMESTAMPs meaning EXTRACT(STRING, STRING) is valid. If EXTRACT(STRING, TIMESTAMP)
   # is added, it takes precedence over the existing EXTRACT(TIMESTAMP, STRING)
   # which could break users.
   [['extract'], 'BIGINT', ['TIMESTAMP', 'STRING'],
-   '_ZN6impala11UdfBuiltins7ExtractEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
-   '_ZN6impala11UdfBuiltins21SwappedExtractPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins12ExtractCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins19ExtractForTimestampEPN10impala_udf15FunctionContextERKNS1_12TimestampValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins26ExtractForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins24ExtractForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
   [['date_part'], 'BIGINT', ['STRING', 'TIMESTAMP'],
-   '_ZN6impala11UdfBuiltins7ExtractEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
-   '_ZN6impala11UdfBuiltins14ExtractPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins12ExtractCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins20DatePartForTimestampEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
+   '_ZN6impala11UdfBuiltins27DatePartForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins25DatePartForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['extract'], 'BIGINT', ['DATE', 'STRING'],
+   '_ZN6impala11UdfBuiltins14ExtractForDateEPN10impala_udf15FunctionContextERKNS1_7DateValERKNS1_9StringValE',
+   '_ZN6impala11UdfBuiltins21ExtractForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins19ExtractForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+  [['date_part'], 'BIGINT', ['STRING', 'DATE'],
+   '_ZN6impala11UdfBuiltins15DatePartForDateEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_7DateValE',
+   '_ZN6impala11UdfBuiltins22DatePartForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins20DatePartForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
 
   [['madlib_encode_vector'], 'STRING', ['STRING'],
     '_ZN6impala11UdfBuiltins12EncodeVectorEPN10impala_udf15FunctionContextERKNS1_9StringValE'],
@@ -127,9 +139,9 @@ visible_functions = [
   [['to_date'], 'STRING', ['TIMESTAMP'], '_ZN6impala18TimestampFunctions6ToDateEPN10impala_udf15FunctionContextERKNS1_12TimestampValE'],
   [['dayname'], 'STRING', ['TIMESTAMP'], '_ZN6impala18TimestampFunctions11LongDayNameEPN10impala_udf15FunctionContextERKNS1_12TimestampValE'],
   [['date_trunc'], 'TIMESTAMP', ['STRING', 'TIMESTAMP'],
-   '_ZN6impala11UdfBuiltins9DateTruncEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
-   '_ZN6impala11UdfBuiltins16DateTruncPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
-   '_ZN6impala11UdfBuiltins14DateTruncCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+   '_ZN6impala11UdfBuiltins21DateTruncForTimestampEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_12TimestampValE',
+   '_ZN6impala11UdfBuiltins28DateTruncForTimestampPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins26DateTruncForTimestampCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
   [['years_add'], 'TIMESTAMP', ['TIMESTAMP', 'INT'],
       '_ZN6impala18TimestampFunctions6AddSubILb1EN10impala_udf6IntValEN5boost9date_time14years_durationINS4_9gregorian21greg_durations_configEEELb0EEENS2_12TimestampValEPNS2_15FunctionContextERKSA_RKT0_'],
   [['years_add'], 'TIMESTAMP', ['TIMESTAMP', 'BIGINT'],
@@ -256,6 +268,12 @@ visible_functions = [
    '_ZN6impala18TimestampFunctions22UnixAndFromUnixPrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
    '_ZN6impala18TimestampFunctions20UnixAndFromUnixCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
 
+  # Date functions
+  [['date_trunc'], 'DATE', ['STRING', 'DATE'],
+   '_ZN6impala11UdfBuiltins16DateTruncForDateEPN10impala_udf15FunctionContextERKNS1_9StringValERKNS1_7DateValE',
+   '_ZN6impala11UdfBuiltins23DateTruncForDatePrepareEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE',
+   '_ZN6impala11UdfBuiltins21DateTruncForDateCloseEPN10impala_udf15FunctionContextENS2_18FunctionStateScopeE'],
+
   # Math builtin functions
   [['pi'], 'DOUBLE', [], 'impala::MathFunctions::Pi'],
   [['e'], 'DOUBLE', [], 'impala::MathFunctions::E'],
diff --git a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
index d732682..3d850aa 100644
--- a/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
+++ b/fe/src/main/java/org/apache/impala/analysis/ExtractFromExpr.java
@@ -36,15 +36,25 @@ public class ExtractFromExpr extends FunctionCallExpr {
 
   // Behaves like an immutable linked hash set containing the TExtractFields in the same
   // order as declared.
-  private static final Set<String> EXTRACT_FIELDS;
+  private static final Set<String> TIMESTAMP_EXTRACT_FIELDS;
+  private static final Set<String> DATE_EXTRACT_FIELDS;
   static {
-    ImmutableSet.Builder<String> builder = new ImmutableSet.Builder<String>();
+    ImmutableSet.Builder<String> timestamp_builder = new ImmutableSet.Builder<String>();
+    ImmutableSet.Builder<String> date_builder = new ImmutableSet.Builder<String>();
     for (TExtractField extractField: TExtractField.values()) {
       if (extractField != TExtractField.INVALID_FIELD) {
-        builder.add(extractField.name());
+        timestamp_builder.add(extractField.name());
+
+        if (extractField != TExtractField.HOUR && extractField != TExtractField.MINUTE
+            && extractField != TExtractField.SECOND
+            && extractField != TExtractField.MILLISECOND
+            && extractField != TExtractField.EPOCH) {
+          date_builder.add(extractField.name());
+        }
       }
     }
-    EXTRACT_FIELDS = builder.build();
+    TIMESTAMP_EXTRACT_FIELDS = timestamp_builder.build();
+    DATE_EXTRACT_FIELDS = date_builder.build();
   }
 
   public ExtractFromExpr(FunctionName fnName, String extractFieldIdent, Expr e) {
@@ -83,10 +93,18 @@ public class ExtractFromExpr extends FunctionCallExpr {
     String extractFieldIdent =
         ((StringLiteral)children_.get(1)).getValueWithOriginalEscapes();
     Preconditions.checkNotNull(extractFieldIdent);
-    if (!EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase())) {
+
+    boolean isDate = children_.get(0).getType().isDate();
+    boolean isExtractFieldValid = isDate ?
+        DATE_EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase()) :
+        TIMESTAMP_EXTRACT_FIELDS.contains(extractFieldIdent.toUpperCase());
+
+    if (!isExtractFieldValid) {
+      String validExtractFields = Joiner.on(", ").join(
+          isDate ? DATE_EXTRACT_FIELDS : TIMESTAMP_EXTRACT_FIELDS);
       throw new AnalysisException("Time unit '" + extractFieldIdent + "' in expression '"
           + toSql() + "' is invalid. Expected one of "
-          + Joiner.on(", ").join(EXTRACT_FIELDS) + ".");
+          + validExtractFields + ".");
     }
   }
 
@@ -94,7 +112,7 @@ public class ExtractFromExpr extends FunctionCallExpr {
   protected String getFunctionNotFoundError(Type[] argTypes) {
     Expr e = children_.get(0);
     return "Expression '" + e.toSql() + "' in '" + toSql() + "' has a return type of "
-          + e.getType().toSql() + " but a TIMESTAMP is required.";
+          + e.getType().toSql() + " but a TIMESTAMP or DATE is required.";
   }
 
   @Override
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 62cb502..2bb4238 100644
--- a/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
+++ b/fe/src/test/java/org/apache/impala/analysis/AnalyzeExprsTest.java
@@ -1866,12 +1866,21 @@ public class AnalyzeExprsTest extends AnalyzerTest {
 
     // Special cases for FROM in function call
     AnalyzesOk("select extract(year from now())");
+    AnalyzesOk("select extract(year from cast(now() as date))");
+    AnalyzesOk("select extract(year from date_col) from functional.date_tbl");
+    AnalyzesOk("select extract(hour from now())");
+    AnalysisError("select extract(hour from cast(now() as date))",
+        "Time unit 'hour' in expression 'EXTRACT(hour FROM CAST(now() AS DATE))' is " +
+        "invalid. Expected one of YEAR, QUARTER, MONTH, DAY.");
     AnalysisError("select extract(foo from now())",
         "Time unit 'foo' in expression 'EXTRACT(foo FROM now())' is invalid. Expected " +
         "one of YEAR, QUARTER, MONTH, DAY, HOUR, MINUTE, SECOND, MILLISECOND, EPOCH.");
+    AnalysisError("select extract(foo from date_col) from functional.date_tbl",
+        "Time unit 'foo' in expression 'EXTRACT(foo FROM date_col)' is " +
+        "invalid. Expected one of YEAR, QUARTER, MONTH, DAY.");
     AnalysisError("select extract(year from 0)",
         "Expression '0' in 'EXTRACT(year FROM 0)' has a return type of TINYINT but a " +
-        "TIMESTAMP is required.");
+        "TIMESTAMP or DATE is required.");
     AnalysisError("select functional.extract(year from now())",
         "Function functional.extract conflicts with the EXTRACT builtin");
     AnalysisError("select date_part(year from now())",
diff --git a/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test b/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test
index f38287b..a8482b1 100644
--- a/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test
+++ b/testdata/workloads/functional-query/queries/QueryTest/hdfs-partitions.test
@@ -178,5 +178,5 @@ BIGINT
 select int_col from functional.alltypes
 where year=date_part('yyyyMMMdd hh:mm:ss', current_timestamp());
 ---- CATCH
-InternalException: invalid extract field: yyyyMMMdd hh:mm:ss
+InternalException: Invalid Date Part Field: yyyyMMMdd hh:mm:ss
 ====