You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@drill.apache.org by dz...@apache.org on 2021/08/14 03:48:36 UTC

[drill] branch master updated: DRILL-7926: Age Function Fix (#2284)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 39b565f  DRILL-7926: Age Function Fix (#2284)
39b565f is described below

commit 39b565f112122734c080324fdcbef518ced16507
Author: dzamo <91...@users.noreply.github.com>
AuthorDate: Sat Aug 14 05:48:27 2021 +0200

    DRILL-7926: Age Function Fix (#2284)
    
    * Fix query start time code and Drill timestamp util functions.
    
    * Implement AGE function as a Freemarker template.
    
    * Add tests for the unary form of AGE.
    
    * Use Java Time's LocalDateTime.until to simplify TIMESTAMPDIFF.
    
    * Fix expressions in the test physical plan in age.json.
    
    * Add comments explaining AGE function logic.
    
    Co-authored-by: Oleg Zinoviev <oz...@solit-clouds.ru>
---
 .../DateIntervalFunctionTemplates/AgeFunction.java |  121 +++
 ...mestampDiff.java => TimestampDiffFunction.java} |   24 +-
 .../drill/exec/expr/fn/impl/DateTypeFunctions.java | 1046 ++++++++++----------
 .../apache/drill/exec/ops/ContextInformation.java  |   19 +-
 .../drill/exec/fn/impl/TestDateFunctions.java      |   17 +-
 .../src/test/resources/functions/date/age.json     |   25 +-
 6 files changed, 691 insertions(+), 561 deletions(-)

diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/AgeFunction.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/AgeFunction.java
new file mode 100644
index 0000000..672311c
--- /dev/null
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/AgeFunction.java
@@ -0,0 +1,121 @@
+/*
+ * 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.
+ */
+<@pp.dropOutputFile />
+<#assign className="GAge"/>
+
+<@pp.changeOutputFile name="/org/apache/drill/exec/expr/fn/impl/${className}.java"/>
+
+<#include "/@includes/license.ftl"/>
+
+package org.apache.drill.exec.expr.fn.impl;
+
+import org.apache.drill.exec.expr.DrillSimpleFunc;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate;
+import org.apache.drill.exec.expr.annotations.FunctionTemplate.NullHandling;
+import javax.inject.Inject;
+import org.apache.drill.exec.expr.annotations.Output;
+import org.apache.drill.exec.expr.annotations.Workspace;
+import org.apache.drill.exec.expr.annotations.Param;
+import org.apache.drill.exec.expr.holders.*;
+import org.apache.drill.exec.record.RecordBatch;
+import org.apache.drill.exec.ops.ContextInformation;
+
+/*
+ * This class is generated using freemarker and the ${.template_name} template.
+ */
+
+public class ${className} {
+
+<#list dateIntervalFunc.dates as fromUnit>
+<#list dateIntervalFunc.dates as toUnit>
+
+  /**
+   * Binary form, returns the interval between `right` and `left`.
+   * Note that this function does not count calendar boundary crossings,
+   * e.g. between yesterday 23:00 and today 01:00 is two hours, not one day.
+   * Modeled on the AGE function in PostgreSQL, see
+   * https://www.postgresql.org/docs/current/functions-datetime.html.
+   */
+  @FunctionTemplate(name = "age",
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class Age${fromUnit}To${toUnit} implements DrillSimpleFunc {
+
+    @Param ${toUnit}Holder left;
+    @Param ${fromUnit}Holder right;
+    @Output IntervalHolder out;
+
+    public void setup() {
+    }
+
+    public void eval() {
+      java.time.LocalDateTime from = java.time.Instant.ofEpochMilli(right.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+      java.time.LocalDateTime to = java.time.Instant.ofEpochMilli(left.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+
+      long months = from.until(to, java.time.temporal.ChronoUnit.MONTHS);
+      from = from.plusMonths(months);
+      long days = from.until(to, java.time.temporal.ChronoUnit.DAYS);
+      from = from.plusDays(days);
+      long millis = from.until(to, java.time.temporal.ChronoUnit.MILLIS);
+
+      out.months = (int) months;
+      out.days = (int) days;
+      out.milliseconds = (int) millis;
+    }
+  }
+</#list>
+  /**
+   * Unary form, subtracts `right` from midnight so equivalent to
+   * `select age(current_date, right)`.
+   * Note that this function does not count calendar boundary crossings,
+   * e.g. between yesterday 23:00 and today 01:00 is two hours, not one day.
+   * Modeled on the AGE function in PostgreSQL, see
+   * https://www.postgresql.org/docs/current/functions-datetime.html.
+   */
+  @FunctionTemplate(name = "age",
+                    scope = FunctionTemplate.FunctionScope.SIMPLE,
+                    nulls = FunctionTemplate.NullHandling.NULL_IF_NULL)
+  public static class Age${fromUnit}ToMidnight implements DrillSimpleFunc {
+
+    @Param ${fromUnit}Holder right;
+    @Workspace java.time.LocalDateTime to;
+    @Output IntervalHolder out;
+    @Inject ContextInformation contextInfo;
+
+    public void setup() {
+      java.time.ZoneId zoneId = contextInfo.getRootFragmentTimeZone();
+      java.time.ZonedDateTime zdtStart = contextInfo.getQueryStartInstant().atZone(zoneId);
+      to = zdtStart.truncatedTo(java.time.temporal.ChronoUnit.DAYS).toLocalDateTime();
+    }
+
+    public void eval() {
+      java.time.LocalDateTime from = java.time.Instant.ofEpochMilli(right.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+
+      long months = from.until(to, java.time.temporal.ChronoUnit.MONTHS);
+      from = from.plusMonths(months);
+      long days = from.until(to, java.time.temporal.ChronoUnit.DAYS);
+      from = from.plusDays(days);
+      long millis = from.until(to, java.time.temporal.ChronoUnit.MILLIS);
+
+      out.months = (int) months;
+      out.days = (int) days;
+      out.milliseconds = (int) millis;
+    }
+  }
+</#list>
+}
diff --git a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/TimestampDiff.java b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/TimestampDiffFunction.java
similarity index 69%
rename from exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/TimestampDiff.java
rename to exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/TimestampDiffFunction.java
index b139efc..3c5e51d 100644
--- a/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/TimestampDiff.java
+++ b/exec/java-exec/src/main/codegen/templates/DateIntervalFunctionTemplates/TimestampDiffFunction.java
@@ -72,25 +72,11 @@ public class ${className} {
     <#elseif unit == "Week">
       out.value = (right.value - left.value) / 604800000; // 7 * 24 * 60 * 60 * 1000
     <#elseif unit == "Month" || unit == "Quarter" || unit == "Year">
-      long timeMilliseconds = left.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis
-          - right.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
-
-      java.time.Period between = java.time.Period.between(
-          java.time.Instant.ofEpochMilli(left.value).atZone(java.time.ZoneOffset.UTC).toLocalDate(),
-          java.time.Instant.ofEpochMilli(right.value).atZone(java.time.ZoneOffset.UTC).toLocalDate());
-      int days = between.getDays();
-      if (timeMilliseconds < 0 && days > 0) {
-        // in the case of negative time value increases left operand days value
-        between = java.time.Period.between(
-            java.time.Instant.ofEpochMilli(left.value + org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis).atZone(java.time.ZoneOffset.UTC).toLocalDate(),
-            java.time.Instant.ofEpochMilli(right.value).atZone(java.time.ZoneOffset.UTC).toLocalDate());
-      } else if (timeMilliseconds > 0 && days < 0) {
-        // in the case of negative days value decreases it for the right operand
-        between = java.time.Period.between(
-            java.time.Instant.ofEpochMilli(left.value - org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis).atZone(java.time.ZoneOffset.UTC).toLocalDate(),
-            java.time.Instant.ofEpochMilli(right.value).atZone(java.time.ZoneOffset.UTC).toLocalDate());
-      }
-      int months = between.getMonths() + between.getYears() * org.apache.drill.exec.vector.DateUtilities.yearsToMonths;
+
+      java.time.LocalDateTime from = java.time.Instant.ofEpochMilli(left.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+      java.time.LocalDateTime to = java.time.Instant.ofEpochMilli(right.value).atZone(java.time.ZoneOffset.UTC).toLocalDateTime();
+
+      int months = (int) from.until(to, java.time.temporal.ChronoUnit.MONTHS);
 
         <#if unit == "Month">
       out.value = months;
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
index afd5f5d..523869a 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/expr/fn/impl/DateTypeFunctions.java
@@ -42,560 +42,542 @@ import io.netty.buffer.DrillBuf;
 
 public class DateTypeFunctions {
 
-    /**
-     * Function to check if a varchar value can be cast to a date.
-     *
-     * At the time of writing this function, several other databases were checked
-     * for behavior compatibility. There was not a consensus between oracle and
-     * Sql server about the expected behavior of this function, and Postgres
-     * lacks it completely.
-     *
-     * Sql Server appears to have both a DATEFORMAT and language locale setting
-     * that can change the values accepted by this function. Oracle appears to
-     * support several formats, some of which are not mentioned in the Sql
-     * Server docs. With the lack of standardization, we decided to implement
-     * this function so that it would only consider date strings that would be
-     * accepted by the cast function as valid.
-     */
-    @SuppressWarnings("unused")
-    @FunctionTemplate(name = "isdate", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.INTERNAL,
-        costCategory = FunctionTemplate.FunctionCostCategory.COMPLEX)
-    public static class IsDate implements DrillSimpleFunc {
-
-      @Param NullableVarCharHolder in;
-      @Output BitHolder out;
-
-      @Override
-      public void setup() { }
-
-      @Override
-      public void eval() {
-        // for a null input return false
-        if (in.isSet == 0) {
-          out.value = 0;
-        } else {
-          out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.isReadableAsDate(in.buffer, in.start, in.end) ? 1 : 0;
-        }
+  /**
+   * Function to check if a varchar value can be cast to a date.
+   *
+   * At the time of writing this function, several other databases were checked
+   * for behavior compatibility. There was not a consensus between oracle and Sql
+   * server about the expected behavior of this function, and Postgres lacks it
+   * completely.
+   *
+   * Sql Server appears to have both a DATEFORMAT and language locale setting that
+   * can change the values accepted by this function. Oracle appears to support
+   * several formats, some of which are not mentioned in the Sql Server docs. With
+   * the lack of standardization, we decided to implement this function so that it
+   * would only consider date strings that would be accepted by the cast function
+   * as valid.
+   */
+  @SuppressWarnings("unused")
+  @FunctionTemplate(name = "isdate", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL, costCategory = FunctionTemplate.FunctionCostCategory.COMPLEX)
+  public static class IsDate implements DrillSimpleFunc {
+
+    @Param
+    NullableVarCharHolder in;
+    @Output
+    BitHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      // for a null input return false
+      if (in.isSet == 0) {
+        out.value = 0;
+      } else {
+        out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.isReadableAsDate(in.buffer,
+            in.start, in.end) ? 1 : 0;
       }
     }
+  }
 
-    // Same as above, just for required input
-    @SuppressWarnings("unused")
-    @FunctionTemplate(name = "isdate", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.INTERNAL,
-        costCategory = FunctionTemplate.FunctionCostCategory.COMPLEX)
-    public static class IsDateRequiredInput implements DrillSimpleFunc {
+  // Same as above, just for required input
+  @SuppressWarnings("unused")
+  @FunctionTemplate(name = "isdate", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.INTERNAL, costCategory = FunctionTemplate.FunctionCostCategory.COMPLEX)
+  public static class IsDateRequiredInput implements DrillSimpleFunc {
 
-      @Param VarCharHolder in;
-      @Output BitHolder out;
+    @Param
+    VarCharHolder in;
+    @Output
+    BitHolder out;
 
-      @Override
-      public void setup() { }
+    @Override
+    public void setup() {
+    }
 
-      @Override
-      public void eval() {
-        // for a null input return false
-        out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.isReadableAsDate(in.buffer, in.start, in.end) ? 1 : 0;
-      }
+    @Override
+    public void eval() {
+      // for a null input return false
+      out.value = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.isReadableAsDate(in.buffer, in.start,
+          in.end) ? 1 : 0;
+    }
+  }
+
+  @FunctionTemplate(name = "intervaltype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class IntervalType implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputYears;
+    @Param
+    BigIntHolder inputMonths;
+    @Param
+    BigIntHolder inputDays;
+    @Param
+    BigIntHolder inputHours;
+    @Param
+    BigIntHolder inputMinutes;
+    @Param
+    BigIntHolder inputSeconds;
+    @Param
+    BigIntHolder inputMilliSeconds;
+    @Output
+    IntervalHolder out;
+
+    @Override
+    public void setup() {
     }
 
-    @FunctionTemplate(name = "intervaltype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class IntervalType implements DrillSimpleFunc {
+    @Override
+    public void eval() {
 
-        @Param  BigIntHolder inputYears;
-        @Param  BigIntHolder inputMonths;
-        @Param  BigIntHolder inputDays;
-        @Param  BigIntHolder inputHours;
-        @Param  BigIntHolder inputMinutes;
-        @Param  BigIntHolder inputSeconds;
-        @Param  BigIntHolder inputMilliSeconds;
-        @Output IntervalHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-
-            out.months       =  (int) ((inputYears.value * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) +
-                                       (inputMonths.value));
-            out.days         =  (int) inputDays.value;
-            out.milliseconds =  (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                                       (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                                       (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                                       (inputMilliSeconds.value));
-        }
-    }
-
-    @FunctionTemplate(name = "interval_year", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class IntervalYearType implements DrillSimpleFunc {
-
-        @Param  BigIntHolder inputYears;
-        @Param  BigIntHolder inputMonths;
-        @Output IntervalYearHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-
-            out.value       = (int) ((inputYears.value * org.apache.drill.exec.vector.DateUtilities.yearsToMonths) +
-                                      (inputMonths.value));
-        }
-    }
-
-    @FunctionTemplate(name = "interval_day", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class IntervalDayType implements DrillSimpleFunc {
-
-        @Param  BigIntHolder inputDays;
-        @Param  BigIntHolder inputHours;
-        @Param  BigIntHolder inputMinutes;
-        @Param  BigIntHolder inputSeconds;
-        @Param  BigIntHolder inputMillis;
-        @Output IntervalDayHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-
-            out.days  = (int) inputDays.value;
-            out.milliseconds =  (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                                       (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                                       (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                                 (inputMillis.value));
-        }
-    }
-
-    @FunctionTemplate(name = "datetype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class DateType implements DrillSimpleFunc {
-
-        @Param  BigIntHolder inputYears;
-        @Param  BigIntHolder inputMonths;
-        @Param  BigIntHolder inputDays;
-        @Output DateHolder   out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            out.value = ((new org.joda.time.MutableDateTime((int) inputYears.value,
-                                                            (int) inputMonths.value,
-                                                            (int)inputDays.value,
-                                                            0,
-                                                            0,
-                                                            0,
-                                                            0,
-                                                            org.joda.time.DateTimeZone.UTC))).getMillis();
-        }
-    }
-
-    @FunctionTemplate(name = "timestamptype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class TimeStampType implements DrillSimpleFunc {
-
-        @Param  BigIntHolder inputYears;
-        @Param  BigIntHolder inputMonths;
-        @Param  BigIntHolder inputDays;
-        @Param  BigIntHolder inputHours;
-        @Param  BigIntHolder inputMinutes;
-        @Param  BigIntHolder inputSeconds;
-        @Param  BigIntHolder inputMilliSeconds;
-        @Output TimeStampHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            out.value = ((new org.joda.time.MutableDateTime((int)inputYears.value,
-                                                            (int)inputMonths.value,
-                                                            (int)inputDays.value,
-                                                            (int)inputHours.value,
-                                                            (int)inputMinutes.value,
-                                                            (int)inputSeconds.value,
-                                                            (int)inputMilliSeconds.value,
-                                                            org.joda.time.DateTimeZone.UTC))).getMillis();
-        }
-    }
-
-    @FunctionTemplate(name = "timetype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class TimeType implements DrillSimpleFunc {
-
-        @Param  BigIntHolder inputHours;
-        @Param  BigIntHolder inputMinutes;
-        @Param  BigIntHolder inputSeconds;
-        @Param  BigIntHolder inputMilliSeconds;
-        @Output TimeHolder   out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            out.value = (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                               (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                               (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                                inputMilliSeconds.value);
-        }
-    }
-
-    @FunctionTemplate(name = "current_date", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isNiladic = true)
-    public static class CurrentDate implements DrillSimpleFunc {
-        @Workspace long queryStartDate;
-        @Output DateHolder out;
-        @Inject ContextInformation contextInfo;
-
-        @Override
-        public void setup() {
-
-            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
-            org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
-            queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).
-                withZoneRetainFields(org.joda.time.DateTimeZone.UTC).getMillis();
-        }
-
-        @Override
-        public void eval() {
-            out.value = queryStartDate;
-        }
-
-    }
-
-    @FunctionTemplate(name = "timeofday", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true,
-                      outputSizeEstimate = OutputSizeEstimateConstants.DATE_TIME_LENGTH)
-    public static class TimeOfDay implements DrillSimpleFunc {
-        @Inject DrillBuf buffer;
-        @Output VarCharHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            java.time.ZonedDateTime temp = java.time.ZonedDateTime.now();
-            String str = org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStampTZ.format(temp);
-            out.buffer = buffer;
-            out.start = 0;
-            out.end = Math.min(100,  str.length()); // truncate if target type has length smaller than that of input's string
-            out.buffer.setBytes(0, str.substring(0,out.end).getBytes());
-        }
-    }
-
-    /*
-     * Return query start time in milliseconds
-     */
-    public static long getQueryStartDate(ContextInformation contextInfo) {
-        org.joda.time.DateTime now = (new org.joda.time.DateTime(contextInfo.getQueryStartTime())).withZoneRetainFields(org.joda.time.DateTimeZone.UTC);
-        return now.getMillis();
-    }
-
-    /*
-     * Niladic version of LocalTimeStamp
-     */
-    @FunctionTemplate(names = {"localtimestamp", "current_timestamp"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isNiladic = true)
-    public static class LocalTimeStampNiladic implements DrillSimpleFunc {
-        @Workspace long queryStartDate;
-        @Output TimeStampHolder out;
-        @Inject ContextInformation contextInfo;
-
-        @Override
-        public void setup() {
-            queryStartDate = org.apache.drill.exec.expr.fn.impl.DateTypeFunctions.getQueryStartDate(contextInfo);
-        }
-
-        @Override
-        public void eval() {
-            out.value = queryStartDate;
-        }
-    }
-
-    /*
-     * Non-Niladic version of LocalTimeStamp
-     */
-    @FunctionTemplate(names = {"now", "statement_timestamp", "transaction_timestamp"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class LocalTimeStampNonNiladic implements DrillSimpleFunc {
-        @Workspace long queryStartDate;
-        @Output TimeStampHolder out;
-        @Inject ContextInformation contextInfo;
-
-        @Override
-        public void setup() {
-            queryStartDate = org.apache.drill.exec.expr.fn.impl.DateTypeFunctions.getQueryStartDate(contextInfo);
-        }
-
-        @Override
-        public void eval() {
-            out.value = queryStartDate;
-        }
-    }
-
-    @FunctionTemplate(names = {"current_time", "localtime"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isNiladic = true)
-    public static class CurrentTime implements DrillSimpleFunc {
-        @Workspace int queryStartTime;
-        @Output TimeHolder out;
-        @Inject ContextInformation contextInfo;
-
-        @Override
-        public void setup() {
-
-            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
-            org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
-            queryStartTime= (now.getHourOfDay() * org.apache.drill.exec.vector.DateUtilities.hoursToMillis) +
-                                   (now.getMinuteOfHour() * org.apache.drill.exec.vector.DateUtilities.minutesToMillis) +
-                                   (now.getSecondOfMinute() * org.apache.drill.exec.vector.DateUtilities.secondsToMillis) +
-                                   (now.getMillisOfSecond());
-        }
-
-        @Override
-        public void eval() {
-            out.value = queryStartTime;
-        }
-    }
-
-    @SuppressWarnings("unused")
-    @FunctionTemplate(names = {"date_add", "add"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
-    public static class DateTimeAddFunction implements DrillSimpleFunc {
-    @Param DateHolder left;
-    @Param TimeHolder right;
-    @Output TimeStampHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            out.value = left.value + right.value;
-        }
-    }
-
-    @SuppressWarnings("unused")
-    @FunctionTemplate(names = {"date_add", "add"}, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
-    public static class TimeDateAddFunction implements DrillSimpleFunc {
-        @Param TimeHolder right;
-        @Param DateHolder left;
-        @Output TimeStampHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            out.value = left.value + right.value;
-        }
-    }
-
-    /* Dummy function template to allow Optiq to validate this function call.
-     * At DrillOptiq time we rewrite all date_part() functions to extract functions,
-     * since they are essentially the same
-     */
-    @SuppressWarnings("unused")
-    @FunctionTemplate(names = "date_part", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls=NullHandling.NULL_IF_NULL)
-    public static class DatePartFunction implements DrillSimpleFunc {
-        @Param VarCharHolder left;
-        @Param DateHolder right;
-        @Output BigIntHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            if (1 == 1) {
-                throw new UnsupportedOperationException("date_part function should be rewritten as extract() functions");
-            }
-        }
-    }
-
-    @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class AgeTimeStampFunction implements DrillSimpleFunc {
-        @Param TimeStampHolder left;
-        @Param TimeStampHolder right;
-        @Output IntervalHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            long diff = left.value - right.value;
-            long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
-            out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-            out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-            out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
-        }
-    }
-
-    @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class AgeTimeStamp2Function implements DrillSimpleFunc {
-        @Param TimeStampHolder right;
-        @Workspace long queryStartDate;
-        @Output IntervalHolder out;
-        @Inject ContextInformation contextInfo;
-
-        @Override
-        public void setup() {
-            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
-            org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
-            queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).getMillis();
-        }
-
-        @Override
-        public void eval() {
-            long diff = queryStartDate - right.value;
-            long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
-            out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-            out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-            out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
-        }
-    }
-
-    @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class AgeDateFunction implements DrillSimpleFunc {
-        @Param DateHolder left;
-        @Param DateHolder right;
-        @Output IntervalHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-          long diff = left.value - right.value;
-          long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
-          out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-          out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-          out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
-        }
-    }
-
-    @FunctionTemplate(name = "age", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class AgeDate2Function implements DrillSimpleFunc {
-        @Param DateHolder right;
-        @Workspace long queryStartDate;
-        @Output IntervalHolder out;
-        @Inject ContextInformation contextInfo;
-
-        @Override
-        public void setup() {
-            int timeZoneIndex = contextInfo.getRootFragmentTimeZone();
-            org.joda.time.DateTimeZone timeZone = org.joda.time.DateTimeZone.forID(org.apache.drill.exec.expr.fn.impl.DateUtility.getTimeZone(timeZoneIndex));
-            org.joda.time.DateTime now = new org.joda.time.DateTime(contextInfo.getQueryStartTime(), timeZone);
-            queryStartDate = (new org.joda.time.DateMidnight(now.getYear(), now.getMonthOfYear(), now.getDayOfMonth(), timeZone)).getMillis();
-        }
-
-        @Override
-        public void eval() {
-            long diff = queryStartDate - right.value;
-            long days = diff / org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis;
-            out.months = (int) (days / org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-            out.days = (int) (days % org.apache.drill.exec.vector.DateUtilities.monthToStandardDays);
-            out.milliseconds = (int) (diff % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
-        }
-    }
-
-    @FunctionTemplate(name = "castTIME", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class CastTimeStampToTime implements DrillSimpleFunc {
-        @Param TimeStampHolder in;
-        @Output TimeHolder out;
-
-        @Override
-        public void setup() {
-        }
-
-        @Override
-        public void eval() {
-            out.value = (int) (in.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
-        }
-    }
-
-    @FunctionTemplate(name = "castTIME", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class CastDateToTime implements DrillSimpleFunc {
-      @Param DateHolder in;
-      @Output TimeHolder out;
-
-      @Override
-      public void setup() {
-      }
+      out.months = (int) ((inputYears.value * org.apache.drill.exec.vector.DateUtilities.yearsToMonths)
+          + (inputMonths.value));
+      out.days = (int) inputDays.value;
+      out.milliseconds = (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis)
+          + (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis)
+          + (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis)
+          + (inputMilliSeconds.value));
+    }
+  }
 
-      @Override
-      public void eval() {
-        out.value = 0;
-      }
+  @FunctionTemplate(name = "interval_year", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class IntervalYearType implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputYears;
+    @Param
+    BigIntHolder inputMonths;
+    @Output
+    IntervalYearHolder out;
+
+    @Override
+    public void setup() {
     }
 
-    @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class UnixTimeStamp implements DrillSimpleFunc {
-      @Output BigIntHolder out;
-      @Workspace long queryStartDate;
-      @Inject ContextInformation contextInfo;
+    @Override
+    public void eval() {
 
-      @Override
-      public void setup() {
-         queryStartDate = contextInfo.getQueryStartTime();
-      }
+      out.value = (int) ((inputYears.value * org.apache.drill.exec.vector.DateUtilities.yearsToMonths)
+          + (inputMonths.value));
+    }
+  }
+
+  @FunctionTemplate(name = "interval_day", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class IntervalDayType implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputDays;
+    @Param
+    BigIntHolder inputHours;
+    @Param
+    BigIntHolder inputMinutes;
+    @Param
+    BigIntHolder inputSeconds;
+    @Param
+    BigIntHolder inputMillis;
+    @Output
+    IntervalDayHolder out;
+
+    @Override
+    public void setup() {
+    }
 
-      @Override
-      public void eval() {
-        out.value = queryStartDate / 1000;
-      }
+    @Override
+    public void eval() {
+
+      out.days = (int) inputDays.value;
+      out.milliseconds = (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis)
+          + (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis)
+          + (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis)
+          + (inputMillis.value));
+    }
+  }
+
+  @FunctionTemplate(name = "datetype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class DateType implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputYears;
+    @Param
+    BigIntHolder inputMonths;
+    @Param
+    BigIntHolder inputDays;
+    @Output
+    DateHolder out;
+
+    @Override
+    public void setup() {
     }
 
-    @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class UnixTimeStampForDate implements DrillSimpleFunc {
-      @Param VarCharHolder inputDateValue;
-      @Output BigIntHolder out;
-      @Workspace org.joda.time.DateTime date;
-      @Workspace org.joda.time.format.DateTimeFormatter formatter;
+    @Override
+    public void eval() {
+      out.value = ((new org.joda.time.MutableDateTime((int) inputYears.value, (int) inputMonths.value,
+          (int) inputDays.value, 0, 0, 0, 0, org.joda.time.DateTimeZone.UTC))).getMillis();
+    }
+  }
+
+  @FunctionTemplate(name = "timestamptype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class TimeStampType implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputYears;
+    @Param
+    BigIntHolder inputMonths;
+    @Param
+    BigIntHolder inputDays;
+    @Param
+    BigIntHolder inputHours;
+    @Param
+    BigIntHolder inputMinutes;
+    @Param
+    BigIntHolder inputSeconds;
+    @Param
+    BigIntHolder inputMilliSeconds;
+    @Output
+    TimeStampHolder out;
+
+    @Override
+    public void setup() {
+    }
 
-      @Override
-      public void setup() {
-         formatter = org.joda.time.format.DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss");
-      }
+    @Override
+    public void eval() {
+      out.value = ((new org.joda.time.MutableDateTime((int) inputYears.value, (int) inputMonths.value,
+          (int) inputDays.value, (int) inputHours.value, (int) inputMinutes.value, (int) inputSeconds.value,
+          (int) inputMilliSeconds.value, org.joda.time.DateTimeZone.UTC))).getMillis();
+    }
+  }
+
+  @FunctionTemplate(name = "timetype", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class TimeType implements DrillSimpleFunc {
+
+    @Param
+    BigIntHolder inputHours;
+    @Param
+    BigIntHolder inputMinutes;
+    @Param
+    BigIntHolder inputSeconds;
+    @Param
+    BigIntHolder inputMilliSeconds;
+    @Output
+    TimeHolder out;
+
+    @Override
+    public void setup() {
+    }
 
-      @Override
-      public void eval() {
-        String inputDate = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(inputDateValue.start, inputDateValue.end, inputDateValue.buffer);
-        date = formatter.parseDateTime(inputDate);
-        out.value = date.getMillis() / 1000;
-      }
+    @Override
+    public void eval() {
+      out.value = (int) ((inputHours.value * org.apache.drill.exec.vector.DateUtilities.hoursToMillis)
+          + (inputMinutes.value * org.apache.drill.exec.vector.DateUtilities.minutesToMillis)
+          + (inputSeconds.value * org.apache.drill.exec.vector.DateUtilities.secondsToMillis)
+          + inputMilliSeconds.value);
+    }
+  }
+
+  @FunctionTemplate(name = "current_date", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isNiladic = true)
+  public static class CurrentDate implements DrillSimpleFunc {
+    @Workspace
+    long queryStartDate;
+    @Output
+    DateHolder out;
+    @Inject
+    ContextInformation contextInfo;
+
+    @Override
+    public void setup() {
+      java.time.Instant queryStart = contextInfo.getQueryStartInstant();
+      java.time.ZonedDateTime lzdt = queryStart.atZone(contextInfo.getRootFragmentTimeZone());
+      java.time.LocalDate ld = lzdt.truncatedTo(java.time.temporal.ChronoUnit.DAYS).toLocalDate();
+      queryStartDate = org.apache.drill.exec.vector.DateUtilities.toDrillDate(ld);
     }
 
-    @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
-    public static class UnixTimeStampForDateWithPattern implements DrillSimpleFunc {
-      @Param VarCharHolder inputDateValue;
-      @Param VarCharHolder inputPattern;
-      @Output BigIntHolder out;
-      @Workspace org.joda.time.DateTime date;
-      @Workspace org.joda.time.format.DateTimeFormatter formatter;
+    @Override
+    public void eval() {
+      out.value = queryStartDate;
+    }
 
-      @Override
-      public void setup() {
-         String pattern = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(inputPattern.start, inputPattern.end, inputPattern.buffer);
-         formatter = org.joda.time.format.DateTimeFormat.forPattern(pattern);
-      }
+  }
+
+  @FunctionTemplate(name = "timeofday", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isRandom = true, outputSizeEstimate = OutputSizeEstimateConstants.DATE_TIME_LENGTH)
+  public static class TimeOfDay implements DrillSimpleFunc {
+    @Inject
+    DrillBuf buffer;
+    @Output
+    VarCharHolder out;
+
+    @Override
+    public void setup() {
+    }
 
-      @Override
-      public void eval() {
-          String inputDate = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers.toStringFromUTF8(inputDateValue.start, inputDateValue.end, inputDateValue.buffer);
-          date = formatter.parseDateTime(inputDate);
-          out.value = date.getMillis() / 1000;
+    @Override
+    public void eval() {
+      java.time.ZonedDateTime temp = java.time.ZonedDateTime.now();
+      String str = org.apache.drill.exec.expr.fn.impl.DateUtility.formatTimeStampTZ.format(temp);
+      out.buffer = buffer;
+      out.start = 0;
+      out.end = Math.min(100, str.length()); // truncate if target type has length smaller than that of input's
+                          // string
+      out.buffer.setBytes(0, str.substring(0, out.end).getBytes());
+    }
+  }
+
+  /*
+   * Return query start time in milliseconds
+   */
+  public static long getQueryStartDate(ContextInformation contextInfo) {
+    org.joda.time.DateTime now = (new org.joda.time.DateTime(contextInfo.getQueryStartTime()))
+        .withZoneRetainFields(org.joda.time.DateTimeZone.UTC);
+    return now.getMillis();
+  }
+
+  /*
+   * Niladic version of LocalTimeStamp
+   */
+  @FunctionTemplate(names = { "localtimestamp",
+      "current_timestamp" }, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isNiladic = true)
+  public static class LocalTimeStampNiladic implements DrillSimpleFunc {
+    @Workspace
+    long queryStartDate;
+    @Output
+    TimeStampHolder out;
+    @Inject
+    ContextInformation contextInfo;
+
+    @Override
+    public void setup() {
+      queryStartDate = org.apache.drill.exec.expr.fn.impl.DateTypeFunctions.getQueryStartDate(contextInfo);
+    }
+
+    @Override
+    public void eval() {
+      out.value = queryStartDate;
+    }
+  }
+
+  /*
+   * Non-Niladic version of LocalTimeStamp
+   */
+  @FunctionTemplate(names = { "now", "statement_timestamp",
+      "transaction_timestamp" }, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class LocalTimeStampNonNiladic implements DrillSimpleFunc {
+    @Workspace
+    long queryStartDate;
+    @Output
+    TimeStampHolder out;
+    @Inject
+    ContextInformation contextInfo;
+
+    @Override
+    public void setup() {
+      queryStartDate = org.apache.drill.exec.expr.fn.impl.DateTypeFunctions.getQueryStartDate(contextInfo);
+    }
+
+    @Override
+    public void eval() {
+      out.value = queryStartDate;
+    }
+  }
+
+  @FunctionTemplate(names = { "current_time",
+      "localtime" }, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL, isNiladic = true)
+  public static class CurrentTime implements DrillSimpleFunc {
+    @Workspace
+    int queryStartTime;
+    @Output
+    TimeHolder out;
+    @Inject
+    ContextInformation contextInfo;
+
+    @Override
+    public void setup() {
+      java.time.Instant queryStart = contextInfo.getQueryStartInstant();
+      java.time.ZonedDateTime lzdt = queryStart.atZone(contextInfo.getRootFragmentTimeZone());
+      queryStartTime = org.apache.drill.exec.vector.DateUtilities.toDrillTime(lzdt.toLocalTime());
+    }
+
+    @Override
+    public void eval() {
+      out.value = queryStartTime;
+    }
+  }
+
+  @SuppressWarnings("unused")
+  @FunctionTemplate(names = { "date_add",
+      "add" }, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class DateTimeAddFunction implements DrillSimpleFunc {
+    @Param
+    DateHolder left;
+    @Param
+    TimeHolder right;
+    @Output
+    TimeStampHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      out.value = left.value + right.value;
+    }
+  }
+
+  @SuppressWarnings("unused")
+  @FunctionTemplate(names = { "date_add",
+      "add" }, scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class TimeDateAddFunction implements DrillSimpleFunc {
+    @Param
+    TimeHolder right;
+    @Param
+    DateHolder left;
+    @Output
+    TimeStampHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      out.value = left.value + right.value;
+    }
+  }
+
+  /*
+   * Dummy function template to allow Optiq to validate this function call. At
+   * DrillOptiq time we rewrite all date_part() functions to extract functions,
+   * since they are essentially the same
+   */
+  @SuppressWarnings("unused")
+  @FunctionTemplate(names = "date_part", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class DatePartFunction implements DrillSimpleFunc {
+    @Param
+    VarCharHolder left;
+    @Param
+    DateHolder right;
+    @Output
+    BigIntHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      if (1 == 1) {
+        throw new UnsupportedOperationException(
+            "date_part function should be rewritten as extract() functions");
       }
     }
+  }
+
+  @FunctionTemplate(name = "castTIME", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class CastTimeStampToTime implements DrillSimpleFunc {
+    @Param
+    TimeStampHolder in;
+    @Output
+    TimeHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      out.value = (int) (in.value % org.apache.drill.exec.vector.DateUtilities.daysToStandardMillis);
+    }
+  }
+
+  @FunctionTemplate(name = "castTIME", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class CastDateToTime implements DrillSimpleFunc {
+    @Param
+    DateHolder in;
+    @Output
+    TimeHolder out;
+
+    @Override
+    public void setup() {
+    }
+
+    @Override
+    public void eval() {
+      out.value = 0;
+    }
+  }
+
+  @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class UnixTimeStamp implements DrillSimpleFunc {
+    @Output
+    BigIntHolder out;
+    @Workspace
+    long queryStartDate;
+    @Inject
+    ContextInformation contextInfo;
+
+    @Override
+    public void setup() {
+      queryStartDate = contextInfo.getQueryStartTime();
+    }
+
+    @Override
+    public void eval() {
+      out.value = queryStartDate / 1000;
+    }
+  }
+
+  @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class UnixTimeStampForDate implements DrillSimpleFunc {
+    @Param
+    VarCharHolder inputDateValue;
+    @Output
+    BigIntHolder out;
+    @Workspace
+    org.joda.time.DateTime date;
+    @Workspace
+    org.joda.time.format.DateTimeFormatter formatter;
+
+    @Override
+    public void setup() {
+      formatter = org.joda.time.format.DateTimeFormat.forPattern("yyyy-MM-dd HH:mm:ss");
+    }
+
+    @Override
+    public void eval() {
+      String inputDate = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers
+          .toStringFromUTF8(inputDateValue.start, inputDateValue.end, inputDateValue.buffer);
+      date = formatter.parseDateTime(inputDate);
+      out.value = date.getMillis() / 1000;
+    }
+  }
+
+  @FunctionTemplate(name = "unix_timestamp", scope = FunctionTemplate.FunctionScope.SIMPLE, nulls = NullHandling.NULL_IF_NULL)
+  public static class UnixTimeStampForDateWithPattern implements DrillSimpleFunc {
+    @Param
+    VarCharHolder inputDateValue;
+    @Param
+    VarCharHolder inputPattern;
+    @Output
+    BigIntHolder out;
+    @Workspace
+    org.joda.time.DateTime date;
+    @Workspace
+    org.joda.time.format.DateTimeFormatter formatter;
+
+    @Override
+    public void setup() {
+      String pattern = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers
+          .toStringFromUTF8(inputPattern.start, inputPattern.end, inputPattern.buffer);
+      formatter = org.joda.time.format.DateTimeFormat.forPattern(pattern);
+    }
+
+    @Override
+    public void eval() {
+      String inputDate = org.apache.drill.exec.expr.fn.impl.StringFunctionHelpers
+          .toStringFromUTF8(inputDateValue.start, inputDateValue.end, inputDateValue.buffer);
+      date = formatter.parseDateTime(inputDate);
+      out.value = date.getMillis() / 1000;
+    }
+  }
 }
diff --git a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
index 1972cb7..24fcd10 100644
--- a/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
+++ b/exec/java-exec/src/main/java/org/apache/drill/exec/ops/ContextInformation.java
@@ -19,6 +19,9 @@ package org.apache.drill.exec.ops;
 
 import org.apache.drill.exec.proto.BitControl.QueryContextInformation;
 import org.apache.drill.exec.proto.UserBitShared.UserCredentials;
+import java.time.Instant;
+import java.time.ZoneId;
+import org.apache.drill.exec.expr.fn.impl.DateUtility;
 
 /**
  * Provides query context information (such as query start time, query user, default schema etc.) for UDFs.
@@ -53,17 +56,25 @@ public class ContextInformation {
   }
 
   /**
-   * @return Query start time in milliseconds
+   * @return Query start time in Unix time (ms)
    */
   public long getQueryStartTime() {
     return queryStartTime;
   }
 
   /**
-   * @return Time zone.
+   * @return Query start time as an Instant
    */
-  public int getRootFragmentTimeZone() {
-    return rootFragmentTimeZone;
+  public Instant getQueryStartInstant() {
+    return Instant.ofEpochMilli(queryStartTime);
+  }
+
+  /**
+   * @return Query time zone as a ZoneId
+   */
+  public ZoneId getRootFragmentTimeZone() {
+    String zoneId = DateUtility.getTimeZone(rootFragmentTimeZone);
+    return ZoneId.of(zoneId);
   }
 
   /**
diff --git a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
index 8c4bf8e..1c8b079 100644
--- a/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
+++ b/exec/java-exec/src/test/java/org/apache/drill/exec/fn/impl/TestDateFunctions.java
@@ -104,10 +104,19 @@ public class TestDateFunctions extends PopUnitTestBase {
 
   @Test
   public void testAge() throws Exception {
-    String[] expectedResults = {"P109M16DT82800S",
-                                "P172M27D",
-                                "P-172M-27D",
-                                "P-39M-18DT-63573S"};
+    String[] expectedResults = {
+      "P107M30DT82800S",
+      "P108MT3600S",
+      "P170M12D",
+      "P-170M-12D",
+      "P-39M-2DT-63573S",
+      "P-39M-3DT-22827S",
+      "PT3540S",
+      "PT0S",
+      "PT0S",
+      "P-3D",
+      "P2DT84600S"
+    };
     testCommon(expectedResults, "/functions/date/age.json", "/test_simple_date.json");
   }
 
diff --git a/exec/java-exec/src/test/resources/functions/date/age.json b/exec/java-exec/src/test/resources/functions/date/age.json
index 23ebf38..b455cad 100644
--- a/exec/java-exec/src/test/resources/functions/date/age.json
+++ b/exec/java-exec/src/test/resources/functions/date/age.json
@@ -28,18 +28,39 @@
       "ref" : "TS1",
       "expr" : "age(cast('2010-01-01 10:10:10' as timestamp), cast('2001-01-01 11:10:10' as timestamp))"
     }, {
+      "ref" : "TS2",
+      "expr" : "age(cast('2010-01-01 11:10:10' as timestamp), cast('2001-01-01 10:10:10' as timestamp))"
+    }, {
       "ref" : "D1",
       "expr" : "age(cast('2005-07-11' as date), cast('1991-04-29' as date))"
     }, {
       "ref" : "D2",
       "expr" : "age(cast('1991-04-29' as date), cast('2005-07-11' as date))"
     }, {
-      "ref" : "TS2",
+      "ref" : "TS3",
       "expr" : "age(cast('1997-12-17 23:37:54' as timestamp), cast('2001-03-20 17:17:27' as timestamp))"
+    }, {
+      "ref" : "TS4",
+      "expr" : "age(cast('1997-12-17 17:17:27' as timestamp), cast('2001-03-20 23:37:54' as timestamp))"
+    }, {
+      "ref" : "TS5",
+      "expr" : "age(cast('2021-02-01 00:29:00' as timestamp), cast('2021-01-31 23:30:00' as timestamp))"
+    }, {
+      "ref" : "UD1",
+      "expr" : "age(current_date())"
+    }, {
+      "ref" : "UTS1",
+      "expr" : "age(cast(current_date() as timestamp))"
+    }, {
+      "ref" : "UD2",
+      "expr" : "age(add(current_date(), 3))"
+    }, {
+      "ref" : "UTS2",
+      "expr" : "age(subtract(add(subtract(current_date() , cast('P3D' as INTERVALDAY)) , cast('PT3600S' as INTERVALDAY)) , cast('PT1800S' as INTERVALDAY)))"
     } ]
   }, {
     "@id" : 3,
     "child" : 2,
     "pop" : "screen"
   } ]
-}
\ No newline at end of file
+}