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/04/28 06:21:15 UTC

[calcite] 02/03: [CALCITE-4698] Result type of datetime '+' operators and TIMESTAMPADD function should be the same as the operand type

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 07a29b607199dfb9af46380faecd7747f8526272
Author: Sergey Nuyanzin <sn...@gmail.com>
AuthorDate: Sat Mar 11 22:26:44 2023 +0100

    [CALCITE-4698] Result type of datetime '+' operators and TIMESTAMPADD function should be the same as the operand type
    
    Close apache/calcite#3104
---
 .../calcite/rel/type/RelDataTypeFactoryImpl.java   | 27 +++++++-
 .../calcite/sql/fun/SqlTimestampAddFunction.java   | 47 +++++++-------
 .../calcite/sql/type/SqlTypeFactoryImpl.java       |  9 +--
 .../apache/calcite/sql/test/ResultCheckers.java    | 15 +++++
 .../org/apache/calcite/test/SqlOperatorTest.java   | 74 ++++++++++++++++++++--
 5 files changed, 138 insertions(+), 34 deletions(-)

diff --git a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
index 3bf1a902e8..d73339c50e 100644
--- a/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/type/RelDataTypeFactoryImpl.java
@@ -18,6 +18,7 @@ package org.apache.calcite.rel.type;
 
 import org.apache.calcite.linq4j.tree.Primitive;
 import org.apache.calcite.sql.SqlCollation;
+import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.type.ArraySqlType;
 import org.apache.calcite.sql.type.JavaToSqlTypeConversionRules;
 import org.apache.calcite.sql.type.MapSqlType;
@@ -48,6 +49,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Objects;
 
+import static java.util.Objects.requireNonNull;
+
 /**
  * Abstract base for implementations of {@link RelDataTypeFactory}.
  */
@@ -109,7 +112,7 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
 
   /** Creates a type factory. */
   protected RelDataTypeFactoryImpl(RelDataTypeSystem typeSystem) {
-    this.typeSystem = Objects.requireNonNull(typeSystem, "typeSystem");
+    this.typeSystem = requireNonNull(typeSystem, "typeSystem");
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -313,6 +316,26 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
     return new MapSqlType(keyType, valueType, isNullable);
   }
 
+  protected RelDataType leastRestrictiveIntervalDatetimeType(
+      final RelDataType dateTimeType, final RelDataType type1) {
+    assert SqlTypeUtil.isDatetime(dateTimeType);
+    if (SqlTypeUtil.isIntType(type1)) {
+      return dateTimeType;
+    }
+    final SqlIntervalQualifier intervalQualifier = type1.getIntervalQualifier();
+    requireNonNull(intervalQualifier, "intervalQualifier");
+    if (!dateTimeType.getSqlTypeName().allowsPrec()
+        || intervalQualifier.useDefaultFractionalSecondPrecision()
+        || intervalQualifier.getFractionalSecondPrecision(typeSystem)
+            <= dateTimeType.getPrecision()) {
+      return dateTimeType;
+    } else {
+      return
+          createSqlType(dateTimeType.getSqlTypeName(),
+              intervalQualifier.getFractionalSecondPrecision(typeSystem));
+    }
+  }
+
   // copy a non-record type, setting nullability
   private RelDataType copySimpleType(
       RelDataType type,
@@ -374,7 +397,7 @@ public abstract class RelDataTypeFactoryImpl implements RelDataTypeFactory {
   @Override public RelDataType createTypeWithNullability(
       final RelDataType type,
       final boolean nullable) {
-    Objects.requireNonNull(type, "type");
+    requireNonNull(type, "type");
     RelDataType newType;
     if (type.isNullable() == nullable) {
       newType = type;
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
index 7cef98c097..08bf7ce0f5 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
@@ -32,8 +32,12 @@ import org.apache.calcite.sql.type.SqlTypeTransforms;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 
+import com.google.common.collect.ImmutableMap;
+
 import org.checkerframework.checker.nullness.qual.Nullable;
 
+import java.util.Map;
+
 import static org.apache.calcite.util.Util.first;
 
 /**
@@ -64,8 +68,10 @@ import static org.apache.calcite.util.Util.first;
  */
 public class SqlTimestampAddFunction extends SqlFunction {
 
-  private static final int MILLISECOND_PRECISION = 3;
-  private static final int MICROSECOND_PRECISION = 6;
+  private static final Map<TimeUnit, Integer> FRAC_SECOND_PRECISION_MAP =
+      ImmutableMap.of(TimeUnit.MILLISECOND, 3,
+          TimeUnit.MICROSECOND, 6,
+          TimeUnit.NANOSECOND, 9);
 
   private static final SqlReturnTypeInference RETURN_TYPE_INFERENCE =
       opBinding ->
@@ -84,35 +90,28 @@ public class SqlTimestampAddFunction extends SqlFunction {
 
   static RelDataType deduceType(RelDataTypeFactory typeFactory,
       @Nullable TimeUnit timeUnit, RelDataType datetimeType) {
-    TimeUnit timeUnit2 = first(timeUnit, TimeUnit.EPOCH);
+    final TimeUnit timeUnit2 = first(timeUnit, TimeUnit.EPOCH);
+    SqlTypeName typeName = datetimeType.getSqlTypeName();
     switch (timeUnit2) {
-    case MILLISECOND:
-      return typeFactory.createSqlType(SqlTypeName.TIMESTAMP,
-          MILLISECOND_PRECISION);
-
     case MICROSECOND:
-      return typeFactory.createSqlType(SqlTypeName.TIMESTAMP,
-          MICROSECOND_PRECISION);
-
+    case MILLISECOND:
+    case NANOSECOND:
+      return typeFactory.createSqlType(typeName,
+          Math.max(FRAC_SECOND_PRECISION_MAP.getOrDefault(timeUnit2, 0),
+              datetimeType.getPrecision()));
     case HOUR:
     case MINUTE:
     case SECOND:
-      SqlTypeName typeName = datetimeType.getSqlTypeName();
-      switch (typeName) {
-      case TIME:
-      case TIMESTAMP_WITH_LOCAL_TIME_ZONE:
-        break;
-      default:
-        // If it is not a TIMESTAMP_WITH_LOCAL_TIME_ZONE, operations involving
-        // HOUR, MINUTE, SECOND with DATE or TIMESTAMP types will result in
-        // TIMESTAMP type.
-        typeName = SqlTypeName.TIMESTAMP;
-        break;
+      if (datetimeType.getFamily() == SqlTypeFamily.TIME) {
+        return datetimeType;
+      } else if (datetimeType.getFamily() == SqlTypeFamily.TIMESTAMP) {
+        return
+            typeFactory.createSqlType(typeName,
+                datetimeType.getPrecision());
+      } else {
+        return typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
       }
-      return typeFactory.createSqlType(typeName);
-
     default:
-    case EPOCH:
       return datetimeType;
     }
   }
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
index ff08400ff8..b414e6787e 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeFactoryImpl.java
@@ -473,7 +473,7 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
         if (types.size() > (i + 1)) {
           RelDataType type1 = types.get(i + 1);
           if (SqlTypeUtil.isDatetime(type1)) {
-            resultType = type1;
+            resultType = leastRestrictiveIntervalDatetimeType(type1, type);
             return createTypeWithNullability(resultType,
                 nullCount > 0 || nullableCount > 0);
           }
@@ -497,9 +497,10 @@ public class SqlTypeFactoryImpl extends RelDataTypeFactoryImpl {
         // datetime +/- interval (or integer) = datetime
         if (types.size() > (i + 1)) {
           RelDataType type1 = types.get(i + 1);
-          if (SqlTypeUtil.isInterval(type1)
-              || SqlTypeUtil.isIntType(type1)) {
-            resultType = type;
+          final boolean isInterval1 = SqlTypeUtil.isInterval(type1);
+          final boolean isInt1 = SqlTypeUtil.isIntType(type1);
+          if (isInterval1 || isInt1) {
+            resultType = leastRestrictiveIntervalDatetimeType(type, type1);
             return createTypeWithNullability(resultType,
                 nullCount > 0 || nullableCount > 0);
           }
diff --git a/testkit/src/main/java/org/apache/calcite/sql/test/ResultCheckers.java b/testkit/src/main/java/org/apache/calcite/sql/test/ResultCheckers.java
index 8960a9b376..69fad877f1 100644
--- a/testkit/src/main/java/org/apache/calcite/sql/test/ResultCheckers.java
+++ b/testkit/src/main/java/org/apache/calcite/sql/test/ResultCheckers.java
@@ -28,6 +28,9 @@ import org.hamcrest.Matcher;
 import java.math.BigDecimal;
 import java.sql.ResultSet;
 import java.sql.Types;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashSet;
@@ -57,6 +60,18 @@ public class ResultCheckers {
         JdbcType.BIG_DECIMAL);
   }
 
+  public static SqlTester.ResultChecker isExactDateTime(LocalDateTime dateTime) {
+    return new MatcherResultChecker<>(
+        is(BigDecimal.valueOf(dateTime.toInstant(ZoneOffset.UTC).toEpochMilli())),
+        JdbcType.BIG_DECIMAL);
+  }
+
+  public static SqlTester.ResultChecker isExactTime(LocalTime time) {
+    return new MatcherResultChecker<>(
+        is((int) (time.toNanoOfDay() / 1000_000)),
+        JdbcType.INTEGER);
+  }
+
   public static SqlTester.ResultChecker isWithin(double value, double delta) {
     return new MatcherResultChecker<>(Matchers.within(value, delta),
         JdbcType.DOUBLE);
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 96285ca7c7..5f8d129ef0 100644
--- a/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
+++ b/testkit/src/main/java/org/apache/calcite/test/SqlOperatorTest.java
@@ -96,6 +96,8 @@ import java.sql.ResultSet;
 import java.sql.SQLException;
 import java.sql.Statement;
 import java.text.SimpleDateFormat;
+import java.time.LocalDateTime;
+import java.time.LocalTime;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
@@ -114,6 +116,8 @@ import static org.apache.calcite.linq4j.tree.Expressions.list;
 import static org.apache.calcite.rel.type.RelDataTypeImpl.NON_NULLABLE_SUFFIX;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.PI;
 import static org.apache.calcite.sql.fun.SqlStdOperatorTable.QUANTIFY_OPERATORS;
+import static org.apache.calcite.sql.test.ResultCheckers.isExactDateTime;
+import static org.apache.calcite.sql.test.ResultCheckers.isExactTime;
 import static org.apache.calcite.sql.test.ResultCheckers.isExactly;
 import static org.apache.calcite.sql.test.ResultCheckers.isNullValue;
 import static org.apache.calcite.sql.test.ResultCheckers.isSet;
@@ -2630,6 +2634,31 @@ public class SqlOperatorTest {
     f.checkScalar("timestamp '2003-08-02 12:54:01' "
             + "- interval '-4 2:4' day to minute",
         "2003-08-06 14:58:01", "TIMESTAMP(0) NOT NULL");
+    f.checkScalar("timestamp '2003-08-02 12:54:01' "
+            + "- interval '0.01' second(1, 3)",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 12, 54, 0, 990_000_000)),
+        "TIMESTAMP(3) NOT NULL");
+    f.checkScalar("timestamp '2003-08-02 12:54:01.000' "
+            + "- interval '1' second",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 12, 54, 0, 0)),
+        "TIMESTAMP(3) NOT NULL");
+    f.checkScalar("timestamp '2003-08-02 12:54:01.123' "
+            + "- interval '1' hour",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 11, 54, 1, 123_000_000)),
+        "TIMESTAMP(3) NOT NULL");
+    f.checkScalar("timestamp with local time zone '2003-08-02 12:54:01' "
+            + "- interval '0.456' second(1, 3)",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 12, 54, 0, 544_000_000)),
+        "TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) NOT NULL");
+    f.checkScalar("time '23:54:01' "
+            + "- interval '0.01' second(1, 3)",
+        isExactTime(LocalTime.of(23, 54, 0, 990_000_000)), "TIME(3) NOT NULL");
+    f.checkScalar("time '23:54:01.123' "
+            + "- interval '1' minute",
+        isExactTime(LocalTime.of(23, 53, 1, 123_000_000)), "TIME(3) NOT NULL");
+    f.checkScalar("date '2003-08-02' "
+            + "- interval '1.123' second(1, 3)",
+        "2003-08-02", "DATE NOT NULL");
 
     // Datetime minus year-month interval
     f.checkScalar("timestamp '2003-08-02 12:54:01' - interval '12' year",
@@ -2931,6 +2960,30 @@ public class SqlOperatorTest {
     f.checkScalar("timestamp '2003-08-02 12:54:01'"
             + " + interval '-4 2:4' day to minute",
         "2003-07-29 10:50:01", "TIMESTAMP(0) NOT NULL");
+    f.checkScalar("interval '0.003' SECOND(1, 3) + timestamp '2003-08-02 12:54:01.001'",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 12, 54, 1, 4_000_000)),
+        "TIMESTAMP(3) NOT NULL");
+    f.checkScalar("timestamp '2003-08-02 12:54:01.000' "
+            + "+ interval '1' second",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 12, 54, 2, 0)),
+        "TIMESTAMP(3) NOT NULL");
+    f.checkScalar("timestamp '2003-08-02 12:54:01.123' "
+            + "+ interval '1' hour",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 13, 54, 1, 123_000_000)),
+        "TIMESTAMP(3) NOT NULL");
+    f.checkScalar("timestamp with local time zone '2003-08-02 12:54:01' "
+            + "+ interval '0.456' second(1, 3)",
+        isExactDateTime(LocalDateTime.of(2003, 8, 2, 12, 54, 1, 456_000_000)),
+        "TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) NOT NULL");
+    f.checkScalar("time '23:54:01' "
+            + "+ interval '0.01' second(1, 3)",
+        isExactTime(LocalTime.of(23, 54, 1, 10_000_000)), "TIME(3) NOT NULL");
+    f.checkScalar("time '23:54:01.123' "
+            + "+ interval '1' minute",
+        isExactTime(LocalTime.of(23, 55, 1, 123_000_000)), "TIME(3) NOT NULL");
+    f.checkScalar("date '2003-08-02' "
+            + "+ interval '1.123' second(1, 3)",
+        "2003-08-02", "DATE NOT NULL");
 
     // Datetime plus year-to-month interval
     f.checkScalar("interval '5-3' year to month + date '2005-03-02'",
@@ -8059,12 +8112,12 @@ public class SqlOperatorTest {
         f.checkScalar("timestampadd(" + s
                 + ", 3000000000, timestamp '2016-02-24 12:42:25')",
             "2016-02-24 12:42:28",
-            "TIMESTAMP(0) NOT NULL"));
+            "TIMESTAMP(3) NOT NULL"));
     NANOSECOND_VARIANTS.forEach(s ->
         f.checkScalar("timestampadd(" + s
                 + ", 2000000000, timestamp '2016-02-24 12:42:25')",
             "2016-02-24 12:42:27",
-            "TIMESTAMP(0) NOT NULL"));
+            "TIMESTAMP(3) NOT NULL"));
     MINUTE_VARIANTS.forEach(s ->
         f.checkScalar("timestampadd(" + s
                 + ", 2, timestamp '2016-02-24 12:42:25')",
@@ -8183,11 +8236,24 @@ public class SqlOperatorTest {
   @Test void testTimestampAddFractionalSeconds() {
     final SqlOperatorFixture f = fixture();
     f.setFor(SqlStdOperatorTable.TIMESTAMP_ADD, VmName.EXPAND);
-    f.checkType(
-        "timestampadd(SQL_TSI_FRAC_SECOND, 2, timestamp '2016-02-24 12:42:25.000000')",
+    f.checkType("timestampadd(SQL_TSI_FRAC_SECOND, 2, "
+            + "timestamp '2016-02-24 12:42:25.000000')",
         // "2016-02-24 12:42:25.000002",
         "TIMESTAMP(3) NOT NULL");
 
+    f.checkType("timestampadd(SQL_TSI_FRAC_SECOND, 2, "
+            + "timestamp with local time zone '2016-02-24 12:42:25.000000')",
+        "TIMESTAMP_WITH_LOCAL_TIME_ZONE(3) NOT NULL");
+
+    f.checkType("timestampadd(SECOND, 2, timestamp '2016-02-24 12:42:25.000')",
+        "TIMESTAMP(3) NOT NULL");
+
+    f.checkType("timestampadd(HOUR, 2, time '12:42:25.000')",
+        "TIME(3) NOT NULL");
+
+    f.checkType("timestampadd(MINUTE, 2, time '12:42:25')",
+        "TIME(0) NOT NULL");
+
     // The following test would correctly return "TIMESTAMP(6) NOT NULL" if max
     // precision were 6 or higher
     assumeTrue(f.getFactory().getTypeFactory().getTypeSystem()