You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by jh...@apache.org on 2023/01/22 03:00:34 UTC

[calcite] branch main updated (d2da6532c9 -> 3801b42c0a)

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

jhyde pushed a change to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git


    from d2da6532c9 Site: Switch PMC Chair to Stamatis Zampetakis
     add 9ff2352466 [CALCITE-5341] Upgrade Calcite to Avatica 1.23.0
     new 0ad974c308 [CALCITE-5414] Use DateTimeUtils to correctly convert between java.sql types and Unix timestamps
     new 3801b42c0a [CALCITE-5342] Refactor SqlFunctions methods lastDay, addMonths, subtractMonths to use DateTimeUtils from Avatica

The 2 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../calcite/adapter/enumerable/RexImpTable.java    |  34 +-
 .../adapter/jdbc/JdbcToEnumerableConverter.java    |   8 +-
 .../org/apache/calcite/rel/type/TimeFrameSet.java  |   5 +-
 .../org/apache/calcite/runtime/SqlFunctions.java   | 421 ++++++++++++---------
 .../org/apache/calcite/util/BuiltInMethod.java     |  21 +-
 .../java/org/apache/calcite/test/JdbcTest.java     |  84 ++++
 .../org/apache/calcite/test/SqlFunctionsTest.java  | 386 ++++++++++++++++---
 gradle.properties                                  |   2 +-
 .../java/org/apache/calcite/test/QuidemTest.java   |   1 +
 9 files changed, 732 insertions(+), 230 deletions(-)


[calcite] 02/02: [CALCITE-5342] Refactor SqlFunctions methods lastDay, addMonths, subtractMonths to use DateTimeUtils from Avatica

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhyde pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 3801b42c0a5237ad7ff6b4c22eec96bcde628034
Author: Julian Hyde <jh...@apache.org>
AuthorDate: Mon Jan 16 20:05:55 2023 -0800

    [CALCITE-5342] Refactor SqlFunctions methods lastDay, addMonths, subtractMonths to use DateTimeUtils from Avatica
---
 .../calcite/adapter/enumerable/RexImpTable.java    |  34 ++++-
 .../org/apache/calcite/rel/type/TimeFrameSet.java  |   5 +-
 .../org/apache/calcite/runtime/SqlFunctions.java   | 151 ++++-----------------
 .../org/apache/calcite/util/BuiltInMethod.java     |   9 +-
 .../org/apache/calcite/test/SqlFunctionsTest.java  |  45 ------
 5 files changed, 63 insertions(+), 181 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
index 2454fe9f8c..c4845a7d29 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
@@ -527,7 +527,8 @@ public class RexImpTable {
       map.put(TIMESTAMP_TRUNC, map.get(FLOOR));
       map.put(TIME_TRUNC, map.get(FLOOR));
 
-      defineMethod(LAST_DAY, "lastDay", NullPolicy.STRICT);
+      map.put(LAST_DAY,
+          new LastDayImplementor("lastDay", BuiltInMethod.LAST_DAY));
       map.put(DAYNAME,
           new PeriodNameImplementor("dayName",
               BuiltInMethod.DAYNAME_WITH_TIMESTAMP,
@@ -2081,6 +2082,37 @@ public class RexImpTable {
     }
   }
 
+  /** Implementor for the {@code LAST_DAY} function. */
+  private static class LastDayImplementor extends MethodNameImplementor {
+    private final BuiltInMethod dateMethod;
+
+    LastDayImplementor(String methodName, BuiltInMethod dateMethod) {
+      super(methodName, NullPolicy.STRICT, false);
+      this.dateMethod = dateMethod;
+    }
+
+    @Override String getVariableName() {
+      return methodName;
+    }
+
+    @Override Expression implementSafe(final RexToLixTranslator translator,
+        final RexCall call, final List<Expression> argValueList) {
+      Expression operand = argValueList.get(0);
+      final RelDataType type = call.operands.get(0).getType();
+      switch (type.getSqlTypeName()) {
+      case TIMESTAMP:
+        operand =
+            Expressions.call(BuiltInMethod.TIMESTAMP_TO_DATE.method, operand);
+        // fall through
+      case DATE:
+        return Expressions.call(dateMethod.method.getDeclaringClass(),
+            dateMethod.method.getName(), operand);
+      default:
+        throw new AssertionError("unknown type " + type);
+      }
+    }
+  }
+
   /** Implementor for the {@code FLOOR} and {@code CEIL} functions. */
   private static class FloorImplementor extends MethodNameImplementor {
     final Method timestampMethod;
diff --git a/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java b/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java
index 52713dc122..07869b8504 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/TimeFrameSet.java
@@ -19,7 +19,6 @@ package org.apache.calcite.rel.type;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.avatica.util.TimeUnitRange;
-import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.util.NameMap;
 import org.apache.calcite.util.TimestampString;
 import org.apache.calcite.util.Util;
@@ -214,7 +213,7 @@ public class TimeFrameSet {
     if (perMonth != null
         && perMonth.getNumerator().equals(BigInteger.ONE)) {
       final int m = perMonth.getDenominator().intValueExact(); // e.g. 12 for YEAR
-      return SqlFunctions.addMonths(date, interval * m);
+      return DateTimeUtils.addMonths(date, interval * m);
     }
 
     // Unknown time frame. Return the original value unchanged.
@@ -235,7 +234,7 @@ public class TimeFrameSet {
     if (perMonth != null
         && perMonth.getNumerator().equals(BigInteger.ONE)) {
       final long m = perMonth.getDenominator().longValueExact(); // e.g. 12 for YEAR
-      return SqlFunctions.addMonths(timestamp, (int) (interval * m));
+      return DateTimeUtils.addMonths(timestamp, (int) (interval * m));
     }
 
     // Unknown time frame. Return the original value unchanged.
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 9a7fa9b9b1..bb617f8cf1 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -84,8 +84,6 @@ import java.util.regex.Pattern;
 import static org.apache.calcite.linq4j.Nullness.castNonNull;
 import static org.apache.calcite.util.Static.RESOURCE;
 
-import static java.lang.Math.floorDiv;
-import static java.lang.Math.floorMod;
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static java.util.Objects.requireNonNull;
 
@@ -403,8 +401,7 @@ public class SqlFunctions {
 
   /** SQL {@code ENDS_WITH(binary, binary)} function. */
   public static boolean endsWith(ByteString s0, ByteString s1) {
-    return s0.length() >= s1.length()
-        && s0.substring(s0.length() - s1.length()).equals(s1);
+    return s0.endsWith(s1);
   }
 
   /** SQL {@code STARTS_WITH(string, string)} function. */
@@ -414,8 +411,7 @@ public class SqlFunctions {
 
   /** SQL {@code STARTS_WITH(binary, binary)} function. */
   public static boolean startsWith(ByteString s0, ByteString s1) {
-    return s0.length() >= s1.length()
-        && s0.substring(0, s1.length()).equals(s1);
+    return s0.startsWith(s1);
   }
 
   /** SQL SUBSTRING(string FROM ...) function. */
@@ -2725,33 +2721,6 @@ public class SqlFunctions {
     return v - remainder;
   }
 
-  /**
-   * SQL {@code LAST_DAY} function.
-   *
-   * @param date days since epoch
-   * @return days of the last day of the month since epoch
-   */
-  public static int lastDay(int date) {
-    int y0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
-    int m0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date);
-    int last = lastDay(y0, m0);
-    return DateTimeUtils.ymdToUnixDate(y0, m0, last);
-  }
-
-  /**
-   * SQL {@code LAST_DAY} function.
-   *
-   * @param timestamp milliseconds from epoch
-   * @return milliseconds of the last day of the month since epoch
-   */
-  public static int lastDay(long timestamp) {
-    int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
-    int y0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
-    int m0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date);
-    int last = lastDay(y0, m0);
-    return DateTimeUtils.ymdToUnixDate(y0, m0, last);
-  }
-
   /**
    * SQL {@code DAYNAME} function, applied to a TIMESTAMP argument.
    *
@@ -2820,21 +2789,32 @@ public class SqlFunctions {
    * @return localDate
    */
   private static LocalDate timeStampToLocalDate(long timestamp) {
-    int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
+    int date = timestampToDate(timestamp);
     return dateToLocalDate(date);
   }
 
+  /** Converts a timestamp (milliseconds since epoch)
+   * to a date (days since epoch). */
+  public static int timestampToDate(long timestamp) {
+    return (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
+  }
+
+  /** Converts a timestamp (milliseconds since epoch)
+   * to a time (milliseconds since midnight). */
+  public static int timestampToTime(long timestamp) {
+    return (int) (timestamp % DateTimeUtils.MILLIS_PER_DAY);
+  }
+
   /** SQL {@code CURRENT_TIMESTAMP} function. */
   @NonDeterministic
   public static long currentTimestamp(DataContext root) {
-    // Cast required for JDK 1.6.
-    return (Long) DataContext.Variable.CURRENT_TIMESTAMP.get(root);
+    return DataContext.Variable.CURRENT_TIMESTAMP.get(root);
   }
 
   /** SQL {@code CURRENT_TIME} function. */
   @NonDeterministic
   public static int currentTime(DataContext root) {
-    int time = (int) (currentTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
+    int time = timestampToTime(currentTimestamp(root));
     if (time < 0) {
       time = (int) (time + DateTimeUtils.MILLIS_PER_DAY);
     }
@@ -2845,8 +2825,8 @@ public class SqlFunctions {
   @NonDeterministic
   public static int currentDate(DataContext root) {
     final long timestamp = currentTimestamp(root);
-    int date = (int) (timestamp / DateTimeUtils.MILLIS_PER_DAY);
-    final int time = (int) (timestamp % DateTimeUtils.MILLIS_PER_DAY);
+    int date = timestampToDate(timestamp);
+    final int time = timestampToTime(timestamp);
     if (time < 0) {
       --date;
     }
@@ -2856,19 +2836,18 @@ public class SqlFunctions {
   /** SQL {@code LOCAL_TIMESTAMP} function. */
   @NonDeterministic
   public static long localTimestamp(DataContext root) {
-    // Cast required for JDK 1.6.
-    return (Long) DataContext.Variable.LOCAL_TIMESTAMP.get(root);
+    return DataContext.Variable.LOCAL_TIMESTAMP.get(root);
   }
 
   /** SQL {@code LOCAL_TIME} function. */
   @NonDeterministic
   public static int localTime(DataContext root) {
-    return (int) (localTimestamp(root) % DateTimeUtils.MILLIS_PER_DAY);
+    return timestampToTime(localTimestamp(root));
   }
 
   @NonDeterministic
   public static TimeZone timeZone(DataContext root) {
-    return (TimeZone) DataContext.Variable.TIME_ZONE.get(root);
+    return DataContext.Variable.TIME_ZONE.get(root);
   }
 
   /** SQL {@code USER} function. */
@@ -2885,7 +2864,7 @@ public class SqlFunctions {
 
   @NonDeterministic
   public static Locale locale(DataContext root) {
-    return (Locale) DataContext.Variable.LOCALE.get(root);
+    return DataContext.Variable.LOCALE.get(root);
   }
 
   /** SQL {@code DATEADD} function applied to a custom time frame.
@@ -3335,90 +3314,6 @@ public class SqlFunctions {
     };
   }
 
-  /** Adds a given number of months to a timestamp, represented as the number
-   * of milliseconds since the epoch. */
-  public static long addMonths(long timestamp, int m) {
-    final long millis = floorMod(timestamp, DateTimeUtils.MILLIS_PER_DAY);
-    timestamp -= millis;
-    final long x =
-        addMonths((int) (timestamp / DateTimeUtils.MILLIS_PER_DAY), m);
-    return x * DateTimeUtils.MILLIS_PER_DAY + millis;
-  }
-
-  /** Adds a given number of months to a date, represented as the number of
-   * days since the epoch. */
-  public static int addMonths(int date, int m) {
-    int y0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
-    int m0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date);
-    int d0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.DAY, date);
-    m0 += m;
-    int deltaYear = floorDiv(m0, 12);
-    y0 += deltaYear;
-    m0 = floorMod(m0, 12);
-    if (m0 == 0) {
-      y0 -= 1;
-      m0 += 12;
-    }
-
-    int last = lastDay(y0, m0);
-    if (d0 > last) {
-      d0 = last;
-    }
-    return DateTimeUtils.ymdToUnixDate(y0, m0, d0);
-  }
-
-  private static int lastDay(int y, int m) {
-    switch (m) {
-    case 2:
-      return y % 4 == 0
-          && (y % 100 != 0
-          || y % 400 == 0)
-          ? 29 : 28;
-    case 4:
-    case 6:
-    case 9:
-    case 11:
-      return 30;
-    default:
-      return 31;
-    }
-  }
-
-  /** Finds the number of months between two dates, each represented as the
-   * number of days since the epoch. */
-  public static int subtractMonths(int date0, int date1) {
-    if (date0 < date1) {
-      return -subtractMonths(date1, date0);
-    }
-    // Start with an estimate.
-    // Since no month has more than 31 days, the estimate is <= the true value.
-    int m = (date0 - date1) / 31;
-    for (;;) {
-      int date2 = addMonths(date1, m);
-      if (date2 >= date0) {
-        return m;
-      }
-      int date3 = addMonths(date1, m + 1);
-      if (date3 > date0) {
-        return m;
-      }
-      ++m;
-    }
-  }
-
-  public static int subtractMonths(long t0, long t1) {
-    final long millis0 = floorMod(t0, DateTimeUtils.MILLIS_PER_DAY);
-    final int d0 = (int) floorDiv(t0 - millis0, DateTimeUtils.MILLIS_PER_DAY);
-    final long millis1 = floorMod(t1, DateTimeUtils.MILLIS_PER_DAY);
-    final int d1 = (int) floorDiv(t1 - millis1, DateTimeUtils.MILLIS_PER_DAY);
-    int x = subtractMonths(d0, d1);
-    final long d2 = addMonths(d1, x);
-    if (d2 == d0 && millis0 < millis1) {
-      --x;
-    }
-    return x;
-  }
-
   /**
    * Implements the {@code .} (field access) operator on an object
    * whose type is not known until runtime.
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index ff9c5fcb30..22a36ebb9a 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -410,9 +410,9 @@ public enum BuiltInMethod {
   MULTI_STRING_CONCAT(SqlFunctions.class, "concatMulti", String[].class),
   FLOOR_DIV(Math.class, "floorDiv", long.class, long.class),
   FLOOR_MOD(Math.class, "floorMod", long.class, long.class),
-  ADD_MONTHS(SqlFunctions.class, "addMonths", long.class, int.class),
-  ADD_MONTHS_INT(SqlFunctions.class, "addMonths", int.class, int.class),
-  SUBTRACT_MONTHS(SqlFunctions.class, "subtractMonths", long.class,
+  ADD_MONTHS(DateTimeUtils.class, "addMonths", long.class, int.class),
+  ADD_MONTHS_INT(DateTimeUtils.class, "addMonths", int.class, int.class),
+  SUBTRACT_MONTHS(DateTimeUtils.class, "subtractMonths", long.class,
       long.class),
   FLOOR(SqlFunctions.class, "floor", int.class, int.class),
   CEIL(SqlFunctions.class, "ceil", int.class, int.class),
@@ -528,7 +528,8 @@ public enum BuiltInMethod {
       DataContext.class, String.class, long.class),
   CUSTOM_TIMESTAMP_CEIL(SqlFunctions.class, "customTimestampCeil",
       DataContext.class, String.class, long.class),
-  LAST_DAY(SqlFunctions.class, "lastDay", int.class),
+  TIMESTAMP_TO_DATE(SqlFunctions.class, "timestampToDate", long.class),
+  LAST_DAY(DateTimeUtils.class, "lastDay", int.class),
   DAYNAME_WITH_TIMESTAMP(SqlFunctions.class,
       "dayNameWithTimestamp", long.class, Locale.class),
   DAYNAME_WITH_DATE(SqlFunctions.class,
diff --git a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
index ea1fdc2d6f..9d2c1b89ec 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -39,8 +39,6 @@ import static org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY;
 import static org.apache.calcite.avatica.util.DateTimeUtils.dateStringToUnixDate;
 import static org.apache.calcite.avatica.util.DateTimeUtils.timeStringToUnixDate;
 import static org.apache.calcite.avatica.util.DateTimeUtils.timestampStringToUnixDate;
-import static org.apache.calcite.avatica.util.DateTimeUtils.ymdToUnixDate;
-import static org.apache.calcite.runtime.SqlFunctions.addMonths;
 import static org.apache.calcite.runtime.SqlFunctions.charLength;
 import static org.apache.calcite.runtime.SqlFunctions.concat;
 import static org.apache.calcite.runtime.SqlFunctions.fromBase64;
@@ -57,7 +55,6 @@ import static org.apache.calcite.runtime.SqlFunctions.posixRegex;
 import static org.apache.calcite.runtime.SqlFunctions.regexpReplace;
 import static org.apache.calcite.runtime.SqlFunctions.rtrim;
 import static org.apache.calcite.runtime.SqlFunctions.sha1;
-import static org.apache.calcite.runtime.SqlFunctions.subtractMonths;
 import static org.apache.calcite.runtime.SqlFunctions.toBase64;
 import static org.apache.calcite.runtime.SqlFunctions.toInt;
 import static org.apache.calcite.runtime.SqlFunctions.toIntOptional;
@@ -71,7 +68,6 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.nullValue;
 import static org.hamcrest.MatcherAssert.assertThat;
-import static org.hamcrest.core.AnyOf.anyOf;
 import static org.junit.jupiter.api.Assertions.assertSame;
 import static org.junit.jupiter.api.Assertions.fail;
 
@@ -302,47 +298,6 @@ class SqlFunctionsTest {
     return trim(true, true, " ", s);
   }
 
-  @Test void testAddMonths() {
-    checkAddMonths(2016, 1, 1, 2016, 2, 1, 1);
-    checkAddMonths(2016, 1, 1, 2017, 1, 1, 12);
-    checkAddMonths(2016, 1, 1, 2017, 2, 1, 13);
-    checkAddMonths(2016, 1, 1, 2015, 1, 1, -12);
-    checkAddMonths(2016, 1, 1, 2018, 10, 1, 33);
-    checkAddMonths(2016, 1, 31, 2016, 4, 30, 3);
-    checkAddMonths(2016, 4, 30, 2016, 7, 30, 3);
-    checkAddMonths(2016, 1, 31, 2016, 2, 29, 1);
-    checkAddMonths(2016, 3, 31, 2016, 2, 29, -1);
-    checkAddMonths(2016, 3, 31, 2116, 3, 31, 1200);
-    checkAddMonths(2016, 2, 28, 2116, 2, 28, 1200);
-    checkAddMonths(2019, 9, 1, 2020, 3, 1, 6);
-    checkAddMonths(2019, 9, 1, 2016, 8, 1, -37);
-  }
-
-  private void checkAddMonths(int y0, int m0, int d0, int y1, int m1, int d1,
-      int months) {
-    final int date0 = ymdToUnixDate(y0, m0, d0);
-    final long date = addMonths(date0, months);
-    final int date1 = ymdToUnixDate(y1, m1, d1);
-    assertThat((int) date, is(date1));
-
-    assertThat(subtractMonths(date1, date0),
-        anyOf(is(months), is(months + 1)));
-    assertThat(subtractMonths(date1 + 1, date0),
-        anyOf(is(months), is(months + 1)));
-    assertThat(subtractMonths(date1, date0 + 1),
-        anyOf(is(months), is(months - 1)));
-    assertThat(subtractMonths(d2ts(date1, 1), d2ts(date0, 0)),
-        anyOf(is(months), is(months + 1)));
-    assertThat(subtractMonths(d2ts(date1, 0), d2ts(date0, 1)),
-        anyOf(is(months - 1), is(months), is(months + 1)));
-  }
-
-  /** Converts a date (days since epoch) and milliseconds (since midnight)
-   * into a timestamp (milliseconds since epoch). */
-  private long d2ts(int date, int millis) {
-    return date * DateTimeUtils.MILLIS_PER_DAY + millis;
-  }
-
   @Test void testFloor() {
     checkFloor(0, 10, 0);
     checkFloor(27, 10, 20);


[calcite] 01/02: [CALCITE-5414] Use DateTimeUtils to correctly convert between java.sql types and Unix timestamps

Posted by jh...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

jhyde pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/calcite.git

commit 0ad974c308d74d2de4bb32f3cff66dc08a3bed59
Author: Greg Hart <gr...@protegrity.com>
AuthorDate: Tue Nov 15 10:04:06 2022 -0800

    [CALCITE-5414] Use DateTimeUtils to correctly convert between java.sql types and Unix timestamps
    
    Close apache/calcite#2995
---
 .../adapter/jdbc/JdbcToEnumerableConverter.java    |   8 +-
 .../org/apache/calcite/runtime/SqlFunctions.java   | 270 +++++++++++++---
 .../org/apache/calcite/util/BuiltInMethod.java     |  12 +-
 .../java/org/apache/calcite/test/JdbcTest.java     |  84 +++++
 .../org/apache/calcite/test/SqlFunctionsTest.java  | 341 +++++++++++++++++++++
 .../java/org/apache/calcite/test/QuidemTest.java   |   1 +
 6 files changed, 668 insertions(+), 48 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
index 8f02d040ef..442bc564b5 100644
--- a/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
+++ b/core/src/main/java/org/apache/calcite/adapter/jdbc/JdbcToEnumerableConverter.java
@@ -308,8 +308,12 @@ public class JdbcToEnumerableConverter
     switch (sqlTypeName) {
     case DATE:
       return (nullable
-          ? BuiltInMethod.DATE_TO_INT_OPTIONAL
-          : BuiltInMethod.DATE_TO_INT).method;
+          ? (offset
+          ? BuiltInMethod.DATE_TO_INT_OPTIONAL_OFFSET
+          : BuiltInMethod.DATE_TO_INT_OPTIONAL)
+          : (offset
+              ? BuiltInMethod.DATE_TO_INT_OFFSET
+              : BuiltInMethod.DATE_TO_INT)).method;
     case TIME:
       return (nullable
           ? BuiltInMethod.TIME_TO_INT_OPTIONAL
diff --git a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
index 61d04f0051..9a7fa9b9b1 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -60,6 +60,8 @@ import java.sql.SQLException;
 import java.sql.Timestamp;
 import java.text.DecimalFormat;
 import java.time.LocalDate;
+import java.time.LocalDateTime;
+import java.time.ZoneOffset;
 import java.time.format.DateTimeFormatter;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -67,7 +69,6 @@ import java.util.Base64;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.Comparator;
-import java.util.Date;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.LinkedList;
@@ -93,11 +94,11 @@ import static java.util.Objects.requireNonNull;
  *
  * <p>Not present: and, or, not (builtin operators are better, because they
  * use lazy evaluation. Implementations do not check for null values; the
- * calling code must do that.</p>
+ * calling code must do that.
  *
  * <p>Many of the functions do not check for null values. This is intentional.
  * If null arguments are possible, the code-generation framework checks for
- * nulls before calling the functions.</p>
+ * nulls before calling the functions.
  */
 @SuppressWarnings("UnnecessaryUnboxing")
 @Deterministic
@@ -2086,41 +2087,96 @@ public class SqlFunctions {
         : (Short) cannotConvert(o, short.class);
   }
 
-  /** Converts the Java type used for UDF parameters of SQL DATE type
-   * ({@link java.sql.Date}) to internal representation (int).
+  /**
+   * Converts a SQL DATE value from the Java type
+   * ({@link java.sql.Date}) to the internal representation type
+   * (number of days since January 1st, 1970 as {@code int})
+   * in the local time zone.
+   *
+   * <p>Since a time zone is not available, the date is converted to represent
+   * the same date as a Unix date in UTC as the {@link java.sql.Date} value in
+   * the local time zone.
    *
-   * <p>Converse of {@link #internalToDate(int)}. */
-  public static int toInt(java.util.Date v) {
+   * @see #toInt(java.sql.Date, TimeZone)
+   * @see #internalToDate(int) converse method
+   */
+  public static int toInt(java.sql.Date v) {
     return toInt(v, LOCAL_TZ);
   }
 
-  public static int toInt(java.util.Date v, TimeZone timeZone) {
-    return (int) (toLong(v, timeZone)  / DateTimeUtils.MILLIS_PER_DAY);
+  /**
+   * Converts a SQL DATE value from the Java type
+   * ({@link java.sql.Date}) to the internal representation type
+   * (number of days since January 1st, 1970 as {@code int}).
+   *
+   * <p>The {@link java.sql.Date} class uses the standard Gregorian calendar
+   * which switches from the Julian calendar to the Gregorian calendar in
+   * October 1582. For compatibility with ISO-8601, the internal representation
+   * is converted to use the proleptic Gregorian calendar.
+   *
+   * <p>If the date contains a partial day, it will be rounded to a full day
+   * depending on the milliseconds value. If the milliseconds value is positive,
+   * it will be rounded down to the closest full day. If the milliseconds value
+   * is negative, it will be rounded up to the closest full day.
+   */
+  public static int toInt(java.sql.Date v, TimeZone timeZone) {
+    return DateTimeUtils.sqlDateToUnixDate(v, timeZone);
   }
 
-  public static @PolyNull Integer toIntOptional(java.util.@PolyNull Date v) {
-    return v == null ? castNonNull(null) : toInt(v);
+  /**
+   * Converts a nullable SQL DATE value from the Java type
+   * ({@link java.sql.Date}) to the internal representation type
+   * (number of days since January 1st, 1970 as {@link Integer})
+   * in the local time zone.
+   *
+   * <p>Since a time zone is not available, the date is converted to represent
+   * the same date as a Unix date in UTC as the {@link java.sql.Date} value in
+   * the local time zone.
+   *
+   * @see #toInt(java.sql.Date, TimeZone)
+   * @see #internalToDate(Integer) converse method
+   */
+  public static @PolyNull Integer toIntOptional(java.sql.@PolyNull Date v) {
+    return v == null
+        ? castNonNull(null)
+        : toInt(v);
   }
 
-  public static @PolyNull Integer toIntOptional(java.util.@PolyNull Date v,
+  /**
+   * Converts a nullable SQL DATE value from the Java type
+   * ({@link java.sql.Date}) to the internal representation type
+   * (number of days since January 1st, 1970 as {@link Integer}).
+   *
+   * @see #toInt(java.sql.Date, TimeZone)
+   */
+  public static @PolyNull Integer toIntOptional(java.sql.@PolyNull Date v,
       TimeZone timeZone) {
     return v == null
         ? castNonNull(null)
         : toInt(v, timeZone);
   }
 
-  public static long toLong(Date v) {
-    return toLong(v, LOCAL_TZ);
-  }
-
-  /** Converts the Java type used for UDF parameters of SQL TIME type
-   * ({@link java.sql.Time}) to internal representation (int).
+  /**
+   * Converts a SQL TIME value from the Java type
+   * ({@link java.sql.Time}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@code int})
+   * in the local time zone.
    *
-   * <p>Converse of {@link #internalToTime(int)}. */
+   * @see #toIntOptional(java.sql.Time)
+   * @see #internalToTime(int) converse method
+   */
   public static int toInt(java.sql.Time v) {
-    return (int) (toLong(v) % DateTimeUtils.MILLIS_PER_DAY);
+    return DateTimeUtils.sqlTimeToUnixTime(v, LOCAL_TZ);
   }
 
+  /**
+   * Converts a nullable SQL TIME value from the Java type
+   * ({@link java.sql.Time}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@link Integer}).
+   *
+   * @see #toInt(java.sql.Time)
+   * @see #internalToTime(Integer) converse method
+   */
   public static @PolyNull Integer toIntOptional(java.sql.@PolyNull Time v) {
     return v == null ? castNonNull(null) : toInt(v);
   }
@@ -2137,7 +2193,8 @@ public class SqlFunctions {
     return o instanceof Integer ? (Integer) o
         : o instanceof Number ? toInt((Number) o)
         : o instanceof String ? toInt((String) o)
-        : o instanceof java.util.Date ? toInt((java.util.Date) o)
+        : o instanceof java.sql.Date ? toInt((java.sql.Date) o)
+        : o instanceof java.sql.Time ? toInt((java.sql.Time) o)
         : (Integer) cannotConvert(o, int.class);
   }
 
@@ -2145,27 +2202,80 @@ public class SqlFunctions {
     return o == null ? castNonNull(null) : toInt(o);
   }
 
-  /** Converts the Java type used for UDF parameters of SQL TIMESTAMP type
-   * ({@link java.sql.Timestamp}) to internal representation (long).
+  /**
+   * Converts a SQL TIMESTAMP value from the Java type
+   * ({@link java.util.Date}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@code long}).
    *
-   * <p>Converse of {@link #internalToTimestamp(long)}. */
+   * <p>Since a time zone is not available, converts the timestamp to represent
+   * the same date and time as a Unix timestamp in UTC as the
+   * {@link java.util.Date} value in the local time zone.
+   *
+   * <p>The {@link java.util.Date} class uses the standard Gregorian calendar
+   * which switches from the Julian calendar to the Gregorian calendar in
+   * October 1582. For compatibility with ISO-8601, converts the internal
+   * representation to use the proleptic Gregorian calendar.
+   */
+  public static long toLong(java.util.Date v) {
+    return DateTimeUtils.utilDateToUnixTimestamp(v, LOCAL_TZ);
+  }
+
+  /**
+   * Converts a SQL TIMESTAMP value from the Java type
+   * ({@link Timestamp}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@code long}).
+   *
+   * <p>Since a time zone is not available, converts the timestamp to represent
+   * the same date and time as a Unix timestamp in UTC as the
+   * {@link Timestamp} value in the local time zone.
+   *
+   * @see #toLong(Timestamp, TimeZone)
+   * @see #internalToTimestamp(Long) converse method
+   */
   public static long toLong(Timestamp v) {
     return toLong(v, LOCAL_TZ);
   }
 
-  // mainly intended for java.sql.Timestamp but works for other dates also
-  @SuppressWarnings("JavaUtilDate")
-  public static long toLong(java.util.Date v, TimeZone timeZone) {
-    final long time = v.getTime();
-    return time + timeZone.getOffset(time);
+  /**
+   * Converts a SQL TIMESTAMP value from the Java type
+   * ({@link Timestamp}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@code long}).
+   *
+   * <p>For backwards compatibility, time zone offsets are calculated in
+   * relation to the local time zone instead of UTC. Providing the default time
+   * zone or {@code null} will return the timestamp unmodified.
+   *
+   * <p>The {@link Timestamp} class uses the standard Gregorian calendar which
+   * switches from the Julian calendar to the Gregorian calendar in
+   * October 1582. For compatibility with ISO-8601, the internal representation
+   * is converted to use the proleptic Gregorian calendar.
+   */
+  public static long toLong(Timestamp v, TimeZone timeZone) {
+    return DateTimeUtils.sqlTimestampToUnixTimestamp(v, timeZone);
   }
 
-  // mainly intended for java.sql.Timestamp but works for other dates also
-  public static @PolyNull Long toLongOptional(java.util.@PolyNull Date v) {
+  /**
+   * Converts a nullable SQL TIMESTAMP value from the Java type
+   * ({@link Timestamp}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@link Long})
+   * in the local time zone.
+   *
+   * @see #toLong(Timestamp, TimeZone)
+   * @see #internalToTimestamp(Long) converse method
+   */
+  public static @PolyNull Long toLongOptional(@PolyNull Timestamp v) {
     return v == null ? castNonNull(null) : toLong(v, LOCAL_TZ);
   }
 
-  public static @PolyNull Long toLongOptional(@PolyNull Timestamp v, TimeZone timeZone) {
+  /**
+   * Converts a nullable SQL TIMESTAMP value from the Java type
+   * ({@link Timestamp}) to the internal representation type
+   * (number of milliseconds since January 1st, 1970 as {@link Long}).
+   *
+   * @see #toLong(Timestamp, TimeZone)
+   */
+  public static @PolyNull Long toLongOptional(@PolyNull Timestamp v,
+      TimeZone timeZone) {
     if (v == null) {
       return castNonNull(null);
     }
@@ -2187,6 +2297,9 @@ public class SqlFunctions {
     return o instanceof Long ? (Long) o
         : o instanceof Number ? toLong((Number) o)
         : o instanceof String ? toLong((String) o)
+        : o instanceof java.sql.Date ? toInt((java.sql.Date) o)
+        : o instanceof java.sql.Time ? toInt((java.sql.Time) o)
+        : o instanceof java.sql.Timestamp ? toLong((java.sql.Timestamp) o)
         : o instanceof java.util.Date ? toLong((java.util.Date) o)
         : (Long) cannotConvert(o, long.class);
   }
@@ -2243,24 +2356,61 @@ public class SqlFunctions {
         : toBigDecimal(o.toString());
   }
 
-  /** Converts the internal representation of a SQL DATE (int) to the Java
-   * type used for UDF parameters ({@link java.sql.Date}). */
+  /**
+   * Converts a SQL DATE value from the internal representation type
+   * (number of days since January 1st, 1970) to the Java type
+   * ({@link java.sql.Date}).
+   *
+   * <p>Since a time zone is not available, converts the date to represent the
+   * same date as a {@link java.sql.Date} in the local time zone as the Unix
+   * date in UTC.
+   *
+   * <p>The Unix date should be the number of days since January 1st, 1970
+   * using the proleptic Gregorian calendar as defined by ISO-8601. The
+   * returned {@link java.sql.Date} object will use the standard Gregorian
+   * calendar which switches from the Julian calendar to the Gregorian calendar
+   * in October 1582.
+   *
+   * @see #internalToDate(Integer)
+   * @see #toInt(java.sql.Date) converse method
+   */
   public static java.sql.Date internalToDate(int v) {
-    final long t = v * DateTimeUtils.MILLIS_PER_DAY;
-    return new java.sql.Date(t - LOCAL_TZ.getOffset(t));
+    final LocalDate date = LocalDate.ofEpochDay(v);
+    return java.sql.Date.valueOf(date);
   }
 
-  /** As {@link #internalToDate(int)} but allows nulls. */
+  /**
+   * Converts a nullable SQL DATE value from the internal representation type
+   * (number of days since January 1st, 1970) to the Java type
+   * ({@link java.sql.Date}).
+   *
+   * @see #internalToDate(int)
+   * @see #toIntOptional(java.sql.Date) converse method
+   */
   public static java.sql.@PolyNull Date internalToDate(@PolyNull Integer v) {
     return v == null ? castNonNull(null) : internalToDate(v.intValue());
   }
 
-  /** Converts the internal representation of a SQL TIME (int) to the Java
-   * type used for UDF parameters ({@link java.sql.Time}). */
+  /**
+   * Converts a SQL TIME value from the internal representation type
+   * (number of milliseconds since January 1st, 1970) to the Java type
+   * ({@link java.sql.Time}).
+   *
+   * @see #internalToTime(Integer)
+   * @see #toInt(java.sql.Time) converse method
+   */
   public static java.sql.Time internalToTime(int v) {
     return new java.sql.Time(v - LOCAL_TZ.getOffset(v));
   }
 
+  /**
+   * Converts a nullable SQL TIME value from the internal representation type
+   * (number of milliseconds since January 1st, 1970) to the Java type
+   * ({@link java.sql.Time}).
+   *
+   * @see #internalToTime(Integer)
+   * @see #toIntOptional(java.sql.Time) converse method
+   */
   public static java.sql.@PolyNull Time internalToTime(@PolyNull Integer v) {
     return v == null ? castNonNull(null) : internalToTime(v.intValue());
   }
@@ -2316,12 +2466,48 @@ public class SqlFunctions {
         .toString();
   }
 
-  /** Converts the internal representation of a SQL TIMESTAMP (long) to the Java
-   * type used for UDF parameters ({@link java.sql.Timestamp}). */
+  /**
+   * Converts a SQL TIMESTAMP value from the internal representation type
+   * (number of milliseconds since January 1st, 1970) to the Java Type
+   * ({@link Timestamp})
+   * in the local time zone.
+   *
+   * <p>Since a time zone is not available, the timestamp is converted to
+   * represent the same timestamp as a {@link Timestamp} in the local time zone
+   * as the Unix timestamp in UTC.
+   *
+   * <p>The Unix timestamp should be the number of milliseconds since
+   * January 1st, 1970 using the proleptic Gregorian calendar as defined by
+   * ISO-8601. The returned {@link Timestamp} object will use the standard
+   * Gregorian calendar which switches from the Julian calendar to the
+   * Gregorian calendar in October 1582.
+   *
+   * @see #internalToTimestamp(Long)
+   * @see #toLong(Timestamp, TimeZone)
+   * @see #toLongOptional(Timestamp)
+   * @see #toLongOptional(Timestamp, TimeZone)
+   * @see #toLong(Timestamp) converse method
+   */
   public static java.sql.Timestamp internalToTimestamp(long v) {
-    return new java.sql.Timestamp(v - LOCAL_TZ.getOffset(v));
+    final LocalDateTime dateTime = LocalDateTime.ofEpochSecond(
+        Math.floorDiv(v, DateTimeUtils.MILLIS_PER_SECOND),
+        (int) (Math.floorMod(v, DateTimeUtils.MILLIS_PER_SECOND) * DateTimeUtils.NANOS_PER_MILLI),
+        ZoneOffset.UTC);
+    return java.sql.Timestamp.valueOf(dateTime);
   }
 
+  /**
+   * Converts a nullable SQL TIMESTAMP value from the internal representation
+   * type (number of milliseconds since January 1st, 1970) to the Java Type
+   * ({@link Timestamp})
+   * in the local time zone.
+   *
+   * @see #internalToTimestamp(long)
+   * @see #toLong(Timestamp)
+   * @see #toLong(Timestamp, TimeZone)
+   * @see #toLongOptional(Timestamp, TimeZone)
+   * @see #toLongOptional(Timestamp) converse method
+   */
   public static java.sql.@PolyNull Timestamp internalToTimestamp(@PolyNull Long v) {
     return v == null ? castNonNull(null) : internalToTimestamp(v.longValue());
   }
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index da839ea09d..ff9c5fcb30 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -563,13 +563,17 @@ public enum BuiltInMethod {
       Comparable.class, Comparator.class),
   ROUND_LONG(SqlFunctions.class, "round", long.class, long.class),
   ROUND_INT(SqlFunctions.class, "round", int.class, int.class),
-  DATE_TO_INT(SqlFunctions.class, "toInt", java.util.Date.class),
+  DATE_TO_INT(SqlFunctions.class, "toInt", java.sql.Date.class),
+  DATE_TO_INT_OFFSET(SqlFunctions.class, "toInt", java.sql.Date.class,
+      TimeZone.class),
   DATE_TO_INT_OPTIONAL(SqlFunctions.class, "toIntOptional",
-      java.util.Date.class),
+      java.sql.Date.class),
+  DATE_TO_INT_OPTIONAL_OFFSET(SqlFunctions.class, "toIntOptional",
+      java.sql.Date.class, TimeZone.class),
   TIME_TO_INT(SqlFunctions.class, "toInt", Time.class),
   TIME_TO_INT_OPTIONAL(SqlFunctions.class, "toIntOptional", Time.class),
-  TIMESTAMP_TO_LONG(SqlFunctions.class, "toLong", java.util.Date.class),
-  TIMESTAMP_TO_LONG_OFFSET(SqlFunctions.class, "toLong", java.util.Date.class,
+  TIMESTAMP_TO_LONG(SqlFunctions.class, "toLong", Timestamp.class),
+  TIMESTAMP_TO_LONG_OFFSET(SqlFunctions.class, "toLong", Timestamp.class,
       TimeZone.class),
   TIMESTAMP_TO_LONG_OPTIONAL(SqlFunctions.class, "toLongOptional",
       Timestamp.class),
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index 61f97f90fa..d54a9224a0 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -8030,6 +8030,90 @@ public class JdbcTest {
         .returns("EXPR$0=[0E+1, 1.1]\n");
   }
 
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5414">[CALCITE-5414]</a>
+   * Convert between standard Gregorian and proleptic Gregorian calendars for
+   * literal dates in local time zone. */
+  @Test void testLiteralDateToSqlTimestamp() {
+    CalciteAssert.that()
+        .with(CalciteConnectionProperty.TIME_ZONE, TimeZone.getDefault().getID())
+        .query("select cast('1500-04-30' as date)")
+        .returns(resultSet -> {
+          try {
+            assertTrue(resultSet.next());
+            assertEquals("1500-04-30", resultSet.getString(1));
+            assertEquals(Date.valueOf("1500-04-30"), resultSet.getDate(1));
+            assertFalse(resultSet.next());
+          } catch (SQLException e) {
+            throw new RuntimeException(e);
+          }
+        });
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5414">[CALCITE-5414]</a>
+   * Convert between standard Gregorian and proleptic Gregorian calendars for
+   * literal timestamps in local time zone. */
+  @Test void testLiteralTimestampToSqlTimestamp() {
+    CalciteAssert.that()
+        .with(CalciteConnectionProperty.TIME_ZONE, TimeZone.getDefault().getID())
+        .query("select cast('1500-04-30 12:00:00' as timestamp)")
+        .returns(resultSet -> {
+          try {
+            assertTrue(resultSet.next());
+            assertEquals("1500-04-30 12:00:00", resultSet.getString(1));
+            assertEquals(Timestamp.valueOf("1500-04-30 12:00:00"), resultSet.getTimestamp(1));
+            assertFalse(resultSet.next());
+          } catch (SQLException e) {
+            throw new RuntimeException(e);
+          }
+        });
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5414">[CALCITE-5414]</a>
+   * Convert between standard Gregorian and proleptic Gregorian calendars for
+   * dynamic dates in local time zone. */
+  @Test void testDynamicDateToSqlTimestamp() {
+    final Date date = Date.valueOf("1500-04-30");
+    CalciteAssert.that()
+        .with(CalciteConnectionProperty.TIME_ZONE, TimeZone.getDefault().getID())
+        .query("select cast(? as date)")
+        .consumesPreparedStatement(statement -> statement.setDate(1, date))
+        .returns(resultSet -> {
+          try {
+            assertTrue(resultSet.next());
+            assertEquals("1500-04-30", resultSet.getString(1));
+            assertEquals(date, resultSet.getDate(1));
+            assertFalse(resultSet.next());
+          } catch (SQLException e) {
+            throw new RuntimeException(e);
+          }
+        });
+  }
+
+  /** Test case for
+   * <a href="https://issues.apache.org/jira/browse/CALCITE-5414">[CALCITE-5414]</a>
+   * Convert between standard Gregorian and proleptic Gregorian calendars for
+   * dynamic timestamps in local time zone. */
+  @Test void testDynamicTimestampToSqlTimestamp() {
+    final Timestamp timestamp = Timestamp.valueOf("1500-04-30 12:00:00");
+    CalciteAssert.that()
+        .with(CalciteConnectionProperty.TIME_ZONE, TimeZone.getDefault().getID())
+        .query("select cast(? as timestamp)")
+        .consumesPreparedStatement(statement -> statement.setTimestamp(1, timestamp))
+        .returns(resultSet -> {
+          try {
+            assertTrue(resultSet.next());
+            assertEquals("1500-04-30 12:00:00", resultSet.getString(1));
+            assertEquals(timestamp, resultSet.getTimestamp(1));
+            assertFalse(resultSet.next());
+          } catch (SQLException e) {
+            throw new RuntimeException(e);
+          }
+        });
+  }
+
   private static String sums(int n, boolean c) {
     final StringBuilder b = new StringBuilder();
     for (int i = 0; i < n; i++) {
diff --git a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
index 683333b596..ea1fdc2d6f 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -25,11 +25,20 @@ import org.apache.calcite.runtime.Utilities;
 import org.junit.jupiter.api.Test;
 
 import java.math.BigDecimal;
+import java.sql.Time;
+import java.sql.Timestamp;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Calendar;
 import java.util.Collections;
 import java.util.List;
+import java.util.Locale;
+import java.util.TimeZone;
 
+import static org.apache.calcite.avatica.util.DateTimeUtils.MILLIS_PER_DAY;
+import static org.apache.calcite.avatica.util.DateTimeUtils.dateStringToUnixDate;
+import static org.apache.calcite.avatica.util.DateTimeUtils.timeStringToUnixDate;
+import static org.apache.calcite.avatica.util.DateTimeUtils.timestampStringToUnixDate;
 import static org.apache.calcite.avatica.util.DateTimeUtils.ymdToUnixDate;
 import static org.apache.calcite.runtime.SqlFunctions.addMonths;
 import static org.apache.calcite.runtime.SqlFunctions.charLength;
@@ -37,6 +46,9 @@ import static org.apache.calcite.runtime.SqlFunctions.concat;
 import static org.apache.calcite.runtime.SqlFunctions.fromBase64;
 import static org.apache.calcite.runtime.SqlFunctions.greater;
 import static org.apache.calcite.runtime.SqlFunctions.initcap;
+import static org.apache.calcite.runtime.SqlFunctions.internalToDate;
+import static org.apache.calcite.runtime.SqlFunctions.internalToTime;
+import static org.apache.calcite.runtime.SqlFunctions.internalToTimestamp;
 import static org.apache.calcite.runtime.SqlFunctions.lesser;
 import static org.apache.calcite.runtime.SqlFunctions.lower;
 import static org.apache.calcite.runtime.SqlFunctions.ltrim;
@@ -47,6 +59,10 @@ import static org.apache.calcite.runtime.SqlFunctions.rtrim;
 import static org.apache.calcite.runtime.SqlFunctions.sha1;
 import static org.apache.calcite.runtime.SqlFunctions.subtractMonths;
 import static org.apache.calcite.runtime.SqlFunctions.toBase64;
+import static org.apache.calcite.runtime.SqlFunctions.toInt;
+import static org.apache.calcite.runtime.SqlFunctions.toIntOptional;
+import static org.apache.calcite.runtime.SqlFunctions.toLong;
+import static org.apache.calcite.runtime.SqlFunctions.toLongOptional;
 import static org.apache.calcite.runtime.SqlFunctions.trim;
 import static org.apache.calcite.runtime.SqlFunctions.upper;
 import static org.apache.calcite.test.Matchers.within;
@@ -974,4 +990,329 @@ class SqlFunctionsTest {
       // ok
     }
   }
+
+  /**
+   * Tests that a date in the local time zone converts to a Unix timestamp in
+   * UTC.
+   */
+  @Test void testToIntWithSqlDate() {
+    assertThat(toInt(new java.sql.Date(0L)), is(0));  // rounded to closest day
+    assertThat(sqlDate("1970-01-01"), is(0));
+    assertThat(sqlDate("1500-04-30"), is(dateStringToUnixDate("1500-04-30")));
+  }
+
+  /**
+   * Test calendar conversion from the standard Gregorian calendar used by
+   * {@code java.sql} and the proleptic Gregorian calendar used by Unix
+   * timestamps.
+   */
+  @Test void testToIntWithSqlDateInGregorianShift() {
+    assertThat(sqlDate("1582-10-04"), is(dateStringToUnixDate("1582-10-04")));
+    assertThat(sqlDate("1582-10-05"), is(dateStringToUnixDate("1582-10-15")));
+    assertThat(sqlDate("1582-10-15"), is(dateStringToUnixDate("1582-10-15")));
+  }
+
+  /**
+   * Test date range 0001-01-01 to 9999-12-31 required by ANSI SQL.
+   *
+   * <p>Java may not be able to represent 0001-01-01 depending on the default
+   * time zone. If the date would fall outside of Anno Domini (AD) when
+   * converted to the default time zone, that date should not be tested.
+   *
+   * <p>Not every time zone has a January 1st 12:00am, so this test skips those
+   * dates.
+   */
+  @Test void testToIntWithSqlDateInAnsiDateRange() {
+    for (int i = 2; i <= 9999; ++i) {
+      final String str = String.format(Locale.ROOT, "%04d-01-01", i);
+      final java.sql.Date date = java.sql.Date.valueOf(str);
+      final Timestamp timestamp = new Timestamp(date.getTime());
+      if (timestamp.toString().endsWith("00:00:00.0")) {
+        // Test equality if the time is valid in Java
+        assertThat("Converts '" + str + "' from SQL to Unix date",
+            toInt(date),
+            is(dateStringToUnixDate(str)));
+      } else {
+        // Test result matches legacy behavior if the time cannot be
+        // represented in Java. This probably results in a different date but
+        // is pretty rare.
+        final long expected =
+            (date.getTime() + DateTimeUtils.DEFAULT_ZONE.getOffset(date.getTime()))
+                / DateTimeUtils.MILLIS_PER_DAY;
+        assertThat("Converts '" + str
+                + "' from SQL to Unix date using legacy behavior",
+            toInt(date),
+            is((int) expected));
+      }
+    }
+  }
+
+  /**
+   * Test using a custom {@link TimeZone} to calculate the Unix timestamp.
+   * Dates created by a {@link java.sql.Date} method should be converted
+   * relative to the local time and not UTC.
+   */
+  @Test public void testToIntWithTimeZone() {
+    // Dates created by a Calendar should be converted to a Unix date in that
+    // time zone
+    final Calendar utcCal =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC"), Locale.ROOT);
+    utcCal.set(1970, Calendar.JANUARY, 1, 0, 0, 0);
+    utcCal.set(Calendar.MILLISECOND, 0);
+    assertThat(
+        toInt(new java.sql.Date(utcCal.getTimeInMillis()),
+            utcCal.getTimeZone()),
+        is(0));
+
+    // Dates should be relative to the local time and not UTC
+    final java.sql.Date epoch = java.sql.Date.valueOf("1970-01-01");
+
+    final TimeZone minusDayZone = TimeZone.getDefault();
+    minusDayZone.setRawOffset((int) (minusDayZone.getOffset(0L) - MILLIS_PER_DAY));
+    assertThat(toInt(epoch, minusDayZone), is(-1));
+
+    final TimeZone plusDayZone = TimeZone.getDefault();
+    plusDayZone.setRawOffset((int) (plusDayZone.getOffset(0L) + MILLIS_PER_DAY));
+    assertThat(toInt(epoch, plusDayZone), is(1));
+  }
+
+  /**
+   * Tests that a nullable date in the local time zone converts to a Unix
+   * timestamp in UTC.
+   */
+  @Test void testToIntOptionalWithLocalTimeZone() {
+    assertThat(toIntOptional(java.sql.Date.valueOf("1970-01-01")), is(0));
+    assertThat(toIntOptional((java.sql.Date) null), is(nullValue()));
+  }
+
+  /**
+   * Tests that a nullable date in the given time zone converts to a Unix
+   * timestamp in UTC.
+   */
+  @Test void testToIntOptionalWithCustomTimeZone() {
+    final TimeZone utc = TimeZone.getTimeZone("UTC");
+    assertThat(toIntOptional(new java.sql.Date(0L), utc), is(0));
+    assertThat(toIntOptional(null, utc), is(nullValue()));
+  }
+
+  /**
+   * Tests that a time in the local time zone converts to a Unix time in UTC.
+   */
+  @Test void testToIntWithSqlTime() {
+    assertThat(sqlTime("00:00:00"), is(timeStringToUnixDate("00:00:00")));
+    assertThat(sqlTime("23:59:59"), is(timeStringToUnixDate("23:59:59")));
+  }
+
+  /**
+   * Tests that a nullable time in the local time zone converts to a Unix time
+   * in UTC.
+   */
+  @Test void testToIntOptionalWithSqlTime() {
+    assertThat(toIntOptional(Time.valueOf("00:00:00")), is(0));
+    assertThat(toIntOptional((Time) null), is(nullValue()));
+  }
+
+  /**
+   * Tests that a timestamp in the local time zone converts to a Unix timestamp
+   * in UTC.
+   */
+  @Test void testToLongWithSqlTimestamp() {
+    assertThat(sqlTimestamp("1970-01-01 00:00:00"), is(0L));
+    assertThat(sqlTimestamp("2014-09-30 15:28:27.356"),
+        is(timestampStringToUnixDate("2014-09-30 15:28:27.356")));
+    assertThat(sqlTimestamp("1500-04-30 12:00:00.123"),
+        is(timestampStringToUnixDate("1500-04-30 12:00:00.123")));
+  }
+
+  /**
+   * Test using a custom {@link TimeZone} to calculate the Unix timestamp.
+   * Timestamps created by a {@link Calendar} should be converted to a Unix
+   * timestamp in the given time zone. Timestamps created by a {@link Timestamp}
+   * method should be converted relative to the local time and not UTC.
+   */
+  @Test void testToLongWithSqlTimestampAndCustomTimeZone() {
+    final Timestamp epoch = java.sql.Timestamp.valueOf("1970-01-01 00:00:00");
+
+    final Calendar utcCal =
+        Calendar.getInstance(TimeZone.getTimeZone("UTC"), Locale.ROOT);
+    utcCal.set(1970, Calendar.JANUARY, 1, 0, 0, 0);
+    utcCal.set(Calendar.MILLISECOND, 0);
+    assertThat(toLong(new Timestamp(utcCal.getTimeInMillis()), utcCal.getTimeZone()),
+        is(0L));
+
+    final TimeZone est = TimeZone.getTimeZone("GMT-5:00");
+    assertThat(toLong(epoch, est),
+        is(epoch.getTime() + est.getOffset(epoch.getTime())));
+
+    final TimeZone ist = TimeZone.getTimeZone("GMT+5:00");
+    assertThat(toLong(epoch, ist),
+        is(epoch.getTime() + ist.getOffset(epoch.getTime())));
+  }
+
+  /**
+   * Test calendar conversion from the standard Gregorian calendar used by
+   * {@code java.sql} and the proleptic Gregorian calendar used by Unix
+   * timestamps.
+   */
+  @Test void testToLongWithSqlTimestampInGregorianShift() {
+    assertThat(sqlTimestamp("1582-10-04 00:00:00"),
+        is(timestampStringToUnixDate("1582-10-04 00:00:00")));
+    assertThat(sqlTimestamp("1582-10-05 00:00:00"),
+        is(timestampStringToUnixDate("1582-10-15 00:00:00")));
+    assertThat(sqlTimestamp("1582-10-15 00:00:00"),
+        is(timestampStringToUnixDate("1582-10-15 00:00:00")));
+  }
+
+  /**
+   * Test date range 0001-01-01 to 9999-12-31 required by ANSI SQL.
+   *
+   * <p>Java may not be able to represent 0001-01-01 depending on the default
+   * time zone. If the date would fall outside of Anno Domini (AD) when
+   * converted to the default time zone, that date should not be tested.
+   *
+   * <p>Not every time zone has a January 1st 12:00am, so this test skips those
+   * dates.
+   */
+  @Test void testToLongWithSqlTimestampInAnsiDateRange() {
+    for (int i = 2; i <= 9999; ++i) {
+      final String str = String.format(Locale.ROOT, "%04d-01-01 00:00:00", i);
+      final Timestamp timestamp = Timestamp.valueOf(str);
+      if (timestamp.toString().endsWith("00:00:00.0")) {
+        // Test equality if the time is valid in Java
+        assertThat("Converts '" + str + "' from SQL to Unix timestamp",
+            toLong(timestamp),
+            is(timestampStringToUnixDate(str)));
+      } else {
+        // Test result matches legacy behavior if the time cannot be represented in Java
+        // This probably results in a different date but is pretty rare
+        final long expected = timestamp.getTime()
+            + DateTimeUtils.DEFAULT_ZONE.getOffset(timestamp.getTime());
+        assertThat("Converts '" + str
+                + "' from SQL to Unix timestamp using legacy behavior",
+            toLong(timestamp),
+            is(expected));
+      }
+    }
+  }
+
+  /**
+   * Tests that a nullable timestamp in the local time zone converts to a Unix
+   * timestamp in UTC.
+   */
+  @Test void testToLongOptionalWithLocalTimeZone() {
+    assertThat(toLongOptional(Timestamp.valueOf("1970-01-01 00:00:00")), is(0L));
+    assertThat(toLongOptional(null), is(nullValue()));
+  }
+
+  /**
+   * Tests that a nullable timestamp in the given time zone converts to a Unix
+   * timestamp in UTC.
+   */
+  @Test void testToLongOptionalWithCustomTimeZone() {
+    final TimeZone utc = TimeZone.getTimeZone("UTC");
+    assertThat(toLongOptional(new Timestamp(0L), utc), is(0L));
+    assertThat(toLongOptional(null, utc), is(nullValue()));
+  }
+
+  /**
+   * Tests that a Unix timestamp converts to a date in the local time zone.
+   */
+  @Test void testInternalToDate() {
+    assertThat(internalToDate(0), is(java.sql.Date.valueOf("1970-01-01")));
+    assertThat(internalToDate(dateStringToUnixDate("1500-04-30")),
+        is(java.sql.Date.valueOf("1500-04-30")));
+  }
+
+  /**
+   * Test calendar conversion from the standard Gregorian calendar used by
+   * {@code java.sql} and the proleptic Gregorian calendar used by Unix
+   * timestamps.
+   */
+  @Test void testInternalToDateWithGregorianShift() {
+    // Gregorian shift
+    assertThat(internalToDate(dateStringToUnixDate("1582-10-04")),
+        is(java.sql.Date.valueOf("1582-10-04")));
+    assertThat(internalToDate(dateStringToUnixDate("1582-10-05")),
+        is(java.sql.Date.valueOf("1582-10-15")));
+    assertThat(internalToDate(dateStringToUnixDate("1582-10-15")),
+        is(java.sql.Date.valueOf("1582-10-15")));
+  }
+
+  /**
+   * Test date range 0001-01-01 to 9999-12-31 required by ANSI SQL.
+   *
+   * <p>Java may not be able to represent all dates depending on the default time zone, but both
+   * the expected and actual assertion values handles that in the same way.
+   */
+  @Test void testInternalToDateWithAnsiDateRange() {
+    for (int i = 2; i <= 9999; ++i) {
+      final String str = String.format(Locale.ROOT, "%04d-01-01", i);
+      assertThat(internalToDate(dateStringToUnixDate(str)),
+          is(java.sql.Date.valueOf(str)));
+    }
+  }
+
+  /**
+   * Tests that a Unix time converts to a SQL time in the local time zone.
+   */
+  @Test void testInternalToTime() {
+    assertThat(internalToTime(0), is(Time.valueOf("00:00:00")));
+    assertThat(internalToTime(86399000), is(Time.valueOf("23:59:59")));
+  }
+
+  /**
+   * Tests that a Unix timestamp converts to a SQL timestamp in the local time
+   * zone.
+   */
+  @Test void testInternalToTimestamp() {
+    assertThat(internalToTimestamp(0),
+        is(Timestamp.valueOf("1970-01-01 00:00:00.0")));
+    assertThat(internalToTimestamp(timestampStringToUnixDate("2014-09-30 15:28:27.356")),
+        is(Timestamp.valueOf("2014-09-30 15:28:27.356")));
+    assertThat(internalToTimestamp(timestampStringToUnixDate("1500-04-30 12:00:00.123")),
+        is(Timestamp.valueOf("1500-04-30 12:00:00.123")));
+  }
+
+  /**
+   * Test calendar conversion from the standard Gregorian calendar used by
+   * {@code java.sql} and the proleptic Gregorian calendar used by Unix timestamps.
+   */
+  @Test void testInternalToTimestampWithGregorianShift() {
+    assertThat(
+        internalToTimestamp(timestampStringToUnixDate("1582-10-04 00:00:00")),
+        is(Timestamp.valueOf("1582-10-04 00:00:00.0")));
+    assertThat(
+        internalToTimestamp(timestampStringToUnixDate("1582-10-05 00:00:00")),
+        is(Timestamp.valueOf("1582-10-15 00:00:00.0")));
+    assertThat(
+        internalToTimestamp(timestampStringToUnixDate("1582-10-15 00:00:00")),
+        is(Timestamp.valueOf("1582-10-15 00:00:00.0")));
+  }
+
+  /**
+   * Test date range 0001-01-01 to 9999-12-31 required by ANSI SQL.
+   *
+   * <p>Java may not be able to represent all dates depending on the default
+   * time zone, but both the expected and actual assertion values handles that
+   * in the same way.
+   */
+  @Test void testInternalToTimestampWithAnsiDateRange() {
+    for (int i = 2; i <= 9999; ++i) {
+      final String str = String.format(Locale.ROOT, "%04d-01-01 00:00:00", i);
+      assertThat(internalToTimestamp(timestampStringToUnixDate(str)),
+          is(Timestamp.valueOf(str)));
+    }
+  }
+
+  private int sqlDate(String str) {
+    return toInt(java.sql.Date.valueOf(str));
+  }
+
+  private int sqlTime(String str) {
+    return toInt(java.sql.Time.valueOf(str));
+  }
+
+  private long sqlTimestamp(String str) {
+    return toLong(java.sql.Timestamp.valueOf(str));
+  }
 }
diff --git a/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java b/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java
index debb0b4ff7..555c01f95e 100644
--- a/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/QuidemTest.java
@@ -293,6 +293,7 @@ public abstract class QuidemTest {
             .connect();
       case "catchall":
         return CalciteAssert.that()
+            .with(CalciteConnectionProperty.TIME_ZONE, "UTC")
             .withSchema("s",
                 new ReflectiveSchema(
                     new CatchallSchema()))