You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@calcite.apache.org by ta...@apache.org on 2023/06/08 22:25:38 UTC

[calcite] branch main updated: [CALCITE-5757] BigQuery DATE_TRUNC return type should be ARG0 and TIMESTAMP_TRUNC/DATETIME_TRUNC should return TIMESTAMP for DATE/TIMESTAMPs and TIMESTAMP_LTZ for TIMESTAMP_LTZ

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

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


The following commit(s) were added to refs/heads/main by this push:
     new 7dc94e381b [CALCITE-5757] BigQuery DATE_TRUNC return type should be ARG0 and TIMESTAMP_TRUNC/DATETIME_TRUNC should return TIMESTAMP for DATE/TIMESTAMPs and TIMESTAMP_LTZ for TIMESTAMP_LTZ
7dc94e381b is described below

commit 7dc94e381ba2f4b6da2c110fadc41069c78fdc63
Author: Tanner Clary <ta...@google.com>
AuthorDate: Mon Jun 5 14:36:06 2023 -0700

    [CALCITE-5757] BigQuery DATE_TRUNC return type should be ARG0 and TIMESTAMP_TRUNC/DATETIME_TRUNC should return TIMESTAMP for DATE/TIMESTAMPs and TIMESTAMP_LTZ for TIMESTAMP_LTZ
---
 babel/src/test/resources/sql/big-query.iq          | 26 ++++++++++++++++++++++
 .../calcite/sql/fun/SqlLibraryOperators.java       | 12 +++++-----
 .../org/apache/calcite/sql/type/OperandTypes.java  |  3 +++
 .../org/apache/calcite/sql/type/ReturnTypes.java   | 25 +++++++++++++++++++++
 .../calcite/sql2rel/StandardConvertletTable.java   | 21 +++++++++++++++++
 .../org/apache/calcite/test/SqlOperatorTest.java   | 24 ++++++++++++++++++++
 6 files changed, 105 insertions(+), 6 deletions(-)

diff --git a/babel/src/test/resources/sql/big-query.iq b/babel/src/test/resources/sql/big-query.iq
index ae2ebbf3ef..429ec7b830 100755
--- a/babel/src/test/resources/sql/big-query.iq
+++ b/babel/src/test/resources/sql/big-query.iq
@@ -2462,6 +2462,14 @@ FROM Dates;
 
 !ok
 
+# Show that the return type matches the operand type
+SELECT
+    DATE_TRUNC(TIMESTAMP "2008-12-25 15:30:00", MONTH) AS timestamp_result,
+    DATE_TRUNC(DATETIME "2008-12-25 15:30:00", MONTH) AS datetime_result;
+timestamp_result TIMESTAMP_WITH_LOCAL_TIME_ZONE NOT NULL
+datetime_result TIMESTAMP NOT NULL
+!type
+
 # In the following example, the original date falls on a
 # Sunday. Because the date_part is WEEK(MONDAY), DATE_TRUNC returns
 # the DATE for the preceding Monday.
@@ -2550,6 +2558,15 @@ SELECT
 
 !ok
 
+# Show that the return type matches the operand type, unless it is a date
+# in which case a TIMESTAMP is returned.
+SELECT
+    DATETIME_TRUNC(TIMESTAMP "2008-12-25 15:30:00", MONTH) AS timestamp_result,
+    DATETIME_TRUNC(DATETIME "2008-12-25 15:30:00", MONTH) AS datetime_result;
+timestamp_result TIMESTAMP_WITH_LOCAL_TIME_ZONE NOT NULL
+datetime_result TIMESTAMP NOT NULL
+!type
+
 
 # In the following example, the original DATETIME falls on a
 # Sunday. Because the part is WEEK(MONDAY), DATE_TRUNC returns the
@@ -2679,6 +2696,15 @@ SELECT
 # the result is non-intuitive near daylight savings transitions that
 # are not hour aligned.
 
+# Show that the return type matches the operand type, unless it is a date
+# in which case a TIMESTAMP is returned.
+SELECT
+    TIMESTAMP_TRUNC(TIMESTAMP "2008-12-25 15:30:00", MONTH) AS timestamp_result,
+    TIMESTAMP_TRUNC(DATETIME "2008-12-25 15:30:00", MONTH) AS datetime_result;
+timestamp_result TIMESTAMP_WITH_LOCAL_TIME_ZONE NOT NULL
+datetime_result TIMESTAMP NOT NULL
+!type
+
 # Display of results may differ, depending upon the environment and
 # time zone where this query was executed.
 !if (false) {
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
index 1a09e98e23..e5ea8aa67e 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLibraryOperators.java
@@ -1182,9 +1182,9 @@ public abstract class SqlLibraryOperators {
   @LibraryOperator(libraries = {BIG_QUERY})
   public static final SqlFunction DATE_TRUNC =
       SqlBasicFunction.create("DATE_TRUNC",
-          ReturnTypes.DATE_NULLABLE,
+          ReturnTypes.ARG0_NULLABLE,
           OperandTypes.sequence("'DATE_TRUNC(<DATE>, <DATETIME_INTERVAL>)'",
-              OperandTypes.DATE, OperandTypes.dateInterval()),
+              OperandTypes.DATE_OR_TIMESTAMP, OperandTypes.dateInterval()),
           SqlFunctionCategory.TIMEDATE)
           .withOperandHandler(OperandHandlers.OPERAND_1_MIGHT_BE_TIME_FRAME);
 
@@ -1242,10 +1242,10 @@ public abstract class SqlLibraryOperators {
   @LibraryOperator(libraries = {BIG_QUERY})
   public static final SqlFunction TIMESTAMP_TRUNC =
       SqlBasicFunction.create("TIMESTAMP_TRUNC",
-          ReturnTypes.TIMESTAMP_NULLABLE,
+          ReturnTypes.ARG0_EXCEPT_DATE_NULLABLE,
           OperandTypes.sequence(
               "'TIMESTAMP_TRUNC(<TIMESTAMP>, <DATETIME_INTERVAL>)'",
-              OperandTypes.TIMESTAMP, OperandTypes.timestampInterval()),
+              OperandTypes.DATE_OR_TIMESTAMP, OperandTypes.timestampInterval()),
           SqlFunctionCategory.TIMEDATE);
 
   /** The "DATETIME_TRUNC(timestamp, timeUnit)" function (BigQuery);
@@ -1256,10 +1256,10 @@ public abstract class SqlLibraryOperators {
   @LibraryOperator(libraries = {BIG_QUERY})
   public static final SqlFunction DATETIME_TRUNC =
       SqlBasicFunction.create("DATETIME_TRUNC",
-          ReturnTypes.TIMESTAMP_NULLABLE,
+          ReturnTypes.ARG0_EXCEPT_DATE_NULLABLE,
           OperandTypes.sequence(
               "'DATETIME_TRUNC(<TIMESTAMP>, <DATETIME_INTERVAL>)'",
-              OperandTypes.TIMESTAMP, OperandTypes.timestampInterval()),
+              OperandTypes.DATE_OR_TIMESTAMP, OperandTypes.timestampInterval()),
           SqlFunctionCategory.TIMEDATE);
 
   /** The "TIMESTAMP_SECONDS(bigint)" function; returns a TIMESTAMP value
diff --git a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
index ccafc9eb7a..cd01b9882b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/OperandTypes.java
@@ -426,6 +426,9 @@ public abstract class OperandTypes {
   public static final SqlSingleOperandTypeChecker TIMESTAMP =
       family(SqlTypeFamily.TIMESTAMP);
 
+  public static final SqlSingleOperandTypeChecker DATE_OR_TIMESTAMP =
+      DATE.or(TIMESTAMP);
+
   /** Type-checker that matches "TIMESTAMP WITH LOCAL TIME ZONE" but not other
    * members of the "TIMESTAMP" family (e.g. "TIMESTAMP"). */
   public static final SqlSingleOperandTypeChecker TIMESTAMP_LTZ =
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
index 8fdab7ea51..f0fcbc3ed8 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
@@ -302,6 +302,31 @@ public abstract class ReturnTypes {
   public static final SqlReturnTypeInference DATE_NULLABLE =
       DATE.andThen(SqlTypeTransforms.TO_NULLABLE);
 
+  /**
+   * Type-inference strategy that returns the type of the first operand,
+   * unless it is a DATE, in which case the return type is TIMESTAMP. Supports
+   * cases such as <a href="https://issues.apache.org/jira/browse/CALCITE-5757">[CALCITE-5757]
+   * Incorrect return type for BigQuery TRUNC functions </a>.
+   */
+  public static final SqlReturnTypeInference ARG0_EXCEPT_DATE = opBinding -> {
+    RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+    SqlTypeName op = opBinding.getOperandType(0).getSqlTypeName();
+    switch (op) {
+    case DATE:
+      return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+    default:
+      return typeFactory.createSqlType(op);
+    }
+  };
+
+  /**
+   * Same as {@link #ARG0_EXCEPT_DATE} but returns with nullability if any of
+   * the operands is nullable by using
+   * {@link org.apache.calcite.sql.type.SqlTypeTransforms#TO_NULLABLE}.
+   */
+  public static final SqlReturnTypeInference ARG0_EXCEPT_DATE_NULLABLE =
+      ARG0_EXCEPT_DATE.andThen(SqlTypeTransforms.TO_NULLABLE);
+
   /**
    * Type-inference strategy whereby the result type of a call is TIME(0).
    */
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
index 0ffb53031e..e416f6abbd 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -173,6 +173,11 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
           return e;
         });
 
+    registerOp(SqlLibraryOperators.DATETIME_TRUNC,
+        new TruncConvertlet());
+    registerOp(SqlLibraryOperators.TIMESTAMP_TRUNC,
+        new TruncConvertlet());
+
     registerOp(SqlLibraryOperators.LTRIM,
         new TrimConvertlet(SqlTrimFunction.Flag.LEADING));
     registerOp(SqlLibraryOperators.RTRIM,
@@ -2049,6 +2054,22 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     }
   }
 
+  /** Convertlet that handles the BigQuery {@code DATETIME_TRUNC} and
+   * {@code TIMESTAMP_TRUNC} functions. Ensures that DATE operands are
+   * cast to TIMESTAMPs to match the expected return type for BigQuery. */
+  private static class TruncConvertlet implements SqlRexConvertlet {
+    @Override public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+      final RexBuilder rexBuilder = cx.getRexBuilder();
+      RexNode op1 = cx.convertExpression(call.operand(0));
+      RexNode op2 = cx.convertExpression(call.operand(1));
+      if (op1.getType().getSqlTypeName() == SqlTypeName.DATE) {
+        RelDataType type = cx.getValidator().getValidatedNodeType(call);
+        op1 = cx.getRexBuilder().makeCast(type, op1);
+      }
+      return rexBuilder.makeCall(call.getOperator(), op1, op2);
+    }
+  }
+
   /** Convertlet that handles the BigQuery {@code TIMESTAMP_SUB} function. */
   private static class TimestampSubConvertlet implements SqlRexConvertlet {
     @Override public RexNode convertCall(SqlRexContext cx, SqlCall call) {
diff --git a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
index 500a501573..1ed0e4782b 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -9631,6 +9631,14 @@ public class SqlOperatorTest {
         "2015-02-01 00:00:00", "TIMESTAMP(0) NOT NULL");
     f.checkScalar("timestamp_trunc(timestamp '2015-02-19 12:34:56', year)",
         "2015-01-01 00:00:00", "TIMESTAMP(0) NOT NULL");
+    // verify return type for dates
+    f.checkScalar("timestamp_trunc(date '2008-12-25', month)",
+        "2008-12-01 00:00:00", "TIMESTAMP(0) NOT NULL");
+    f.checkFails("^timestamp_trunc(time '15:30:00', hour)^",
+        "Cannot apply 'TIMESTAMP_TRUNC' to arguments of type "
+            + "'TIMESTAMP_TRUNC\\(<TIME\\(0\\)>, <INTERVAL HOUR>\\)'\\. "
+            + "Supported form\\(s\\): 'TIMESTAMP_TRUNC\\(<TIMESTAMP>, <DATETIME_INTERVAL>\\)'",
+        false);
   }
 
   @Test void testDatetimeTrunc() {
@@ -9669,6 +9677,14 @@ public class SqlOperatorTest {
         "2015-02-01 00:00:00", "TIMESTAMP(0) NOT NULL");
     f.checkScalar("datetime_trunc(timestamp '2015-02-19 12:34:56', year)",
         "2015-01-01 00:00:00", "TIMESTAMP(0) NOT NULL");
+    // verify return type for dates
+    f.checkScalar("datetime_trunc(date '2008-12-25', month)",
+        "2008-12-01 00:00:00", "TIMESTAMP(0) NOT NULL");
+    f.checkFails("^datetime_trunc(time '15:30:00', hour)^",
+        "Cannot apply 'DATETIME_TRUNC' to arguments of type "
+            + "'DATETIME_TRUNC\\(<TIME\\(0\\)>, <INTERVAL HOUR>\\)'\\. "
+            + "Supported form\\(s\\): 'DATETIME_TRUNC\\(<TIMESTAMP>, <DATETIME_INTERVAL>\\)'",
+        false);
   }
 
   @Test void testDateTrunc() {
@@ -9705,6 +9721,14 @@ public class SqlOperatorTest {
         "2015-01-01", "DATE NOT NULL");
     f.checkScalar("date_trunc(date '2015-02-19', isoyear)",
         "2014-12-29", "DATE NOT NULL");
+    // verifies return type for TIME & TIMESTAMP
+    f.checkScalar("date_trunc(timestamp '2008-12-25 15:30:00', month)",
+        "2008-12-01 00:00:00", "TIMESTAMP(0) NOT NULL");
+    f.checkFails("^date_trunc(time '15:30:00', hour)^",
+        "Cannot apply 'DATE_TRUNC' to arguments of type "
+            + "'DATE_TRUNC\\(<TIME\\(0\\)>, <INTERVAL HOUR>\\)'\\. "
+            + "Supported form\\(s\\): 'DATE_TRUNC\\(<DATE>, <DATETIME_INTERVAL>\\)'",
+        false);
   }
 
   @Test void testFormatTime() {