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 2018/06/29 18:42:01 UTC

[5/8] calcite git commit: [CALCITE-2281] Return type of the TIMESTAMPADD function has wrong precision (Sudheesh Katkam)

[CALCITE-2281] Return type of the TIMESTAMPADD function has wrong precision (Sudheesh Katkam)


Project: http://git-wip-us.apache.org/repos/asf/calcite/repo
Commit: http://git-wip-us.apache.org/repos/asf/calcite/commit/44f73389
Tree: http://git-wip-us.apache.org/repos/asf/calcite/tree/44f73389
Diff: http://git-wip-us.apache.org/repos/asf/calcite/diff/44f73389

Branch: refs/heads/master
Commit: 44f7338902627a5306f2390af1b407069bdebdda
Parents: 03b7af4
Author: Sudheesh Katkam <su...@dremio.com>
Authored: Wed Apr 25 17:26:40 2018 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Fri Jun 29 01:23:21 2018 -0700

----------------------------------------------------------------------
 .../sql/fun/SqlDatetimePlusOperator.java        | 18 ++-----
 .../sql/fun/SqlTimestampAddFunction.java        | 50 ++++++++++++++------
 .../sql2rel/StandardConvertletTable.java        | 12 +++--
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 18 +++++++
 4 files changed, 67 insertions(+), 31 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/44f73389/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimePlusOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimePlusOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimePlusOperator.java
index c8f2baf..01e7e53 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimePlusOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimePlusOperator.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.sql.fun;
 
+import org.apache.calcite.avatica.util.TimeUnit;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.sql.SqlCall;
@@ -28,7 +29,6 @@ import org.apache.calcite.sql.type.InferTypes;
 import org.apache.calcite.sql.type.IntervalSqlType;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
-import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 
 /**
@@ -49,21 +49,11 @@ public class SqlDatetimePlusOperator extends SqlSpecialOperator {
     final RelDataType leftType = opBinding.getOperandType(0);
     final IntervalSqlType unitType =
         (IntervalSqlType) opBinding.getOperandType(1);
-    switch (unitType.getIntervalQualifier().getStartUnit()) {
-    case HOUR:
-    case MINUTE:
-    case SECOND:
-    case MILLISECOND:
-    case MICROSECOND:
-      return typeFactory.createTypeWithNullability(
-          typeFactory.createSqlType(SqlTypeName.TIMESTAMP),
-          leftType.isNullable() || unitType.isNullable());
-    default:
-      return leftType;
-    }
+    final TimeUnit timeUnit = unitType.getIntervalQualifier().getStartUnit();
+    return SqlTimestampAddFunction.deduceType(typeFactory, timeUnit,
+        unitType, leftType);
   }
 
-
   public SqlSyntax getSyntax() {
     return SqlSyntax.SPECIAL;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/44f73389/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
----------------------------------------------------------------------
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 a4861f9..6c38ec5 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
@@ -54,28 +54,50 @@ import org.apache.calcite.sql.type.SqlTypeName;
  *
  * <p>Returns modified timestamp.
  */
-class SqlTimestampAddFunction extends SqlFunction {
+public class SqlTimestampAddFunction extends SqlFunction {
+
+  private static final int MILLISECOND_PRECISION = 3;
+  private static final int MICROSECOND_PRECISION = 6;
 
   private static final SqlReturnTypeInference RETURN_TYPE_INFERENCE =
       new SqlReturnTypeInference() {
         public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
           final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
-          switch (opBinding.getOperandLiteralValue(0, TimeUnit.class)) {
-          case HOUR:
-          case MINUTE:
-          case SECOND:
-          case MILLISECOND:
-          case MICROSECOND:
-            return typeFactory.createTypeWithNullability(
-                typeFactory.createSqlType(SqlTypeName.TIMESTAMP),
-                opBinding.getOperandType(1).isNullable()
-                    || opBinding.getOperandType(2).isNullable());
-          default:
-            return opBinding.getOperandType(2);
-          }
+          return deduceType(typeFactory,
+              opBinding.getOperandLiteralValue(0, TimeUnit.class),
+              opBinding.getOperandType(1), opBinding.getOperandType(2));
         }
       };
 
+  public static RelDataType deduceType(RelDataTypeFactory typeFactory,
+      TimeUnit timeUnit, RelDataType operandType1, RelDataType operandType2) {
+    switch (timeUnit) {
+    case HOUR:
+    case MINUTE:
+    case SECOND:
+    case MILLISECOND:
+    case MICROSECOND:
+      final RelDataType type;
+      switch (timeUnit) {
+      case MILLISECOND:
+        type = typeFactory.createSqlType(SqlTypeName.TIMESTAMP,
+            MILLISECOND_PRECISION);
+        break;
+      case MICROSECOND:
+        type = typeFactory.createSqlType(SqlTypeName.TIMESTAMP,
+            MICROSECOND_PRECISION);
+        break;
+      default:
+        type = typeFactory.createSqlType(SqlTypeName.TIMESTAMP);
+      }
+      return typeFactory.createTypeWithNullability(type,
+          operandType1.isNullable()
+              || operandType2.isNullable());
+    default:
+      return operandType2;
+    }
+  }
+
   /** Creates a SqlTimestampAddFunction. */
   SqlTimestampAddFunction() {
     super("TIMESTAMPADD", SqlKind.TIMESTAMP_ADD, RETURN_TYPE_INFERENCE, null,

http://git-wip-us.apache.org/repos/asf/calcite/blob/44f73389/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
----------------------------------------------------------------------
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 fb22286..caf5d5d 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -61,6 +61,7 @@ import org.apache.calcite.sql.fun.SqlOverlapsOperator;
 import org.apache.calcite.sql.fun.SqlRowOperator;
 import org.apache.calcite.sql.fun.SqlSequenceValueOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.fun.SqlTimestampAddFunction;
 import org.apache.calcite.sql.fun.SqlTrimFunction;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
@@ -1376,13 +1377,18 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       final RexBuilder rexBuilder = cx.getRexBuilder();
       final SqlLiteral unitLiteral = call.operand(0);
       final TimeUnit unit = unitLiteral.symbolValue(TimeUnit.class);
+      final RexNode operand1 = cx.convertExpression(call.operand(1));
+      final RexNode operand2 = cx.convertExpression(call.operand(2));
+      final RelDataType type =
+          SqlTimestampAddFunction.deduceType(cx.getTypeFactory(), unit,
+              operand1.getType(), operand2.getType());
+      final RexNode operand2b = rexBuilder.makeCast(type, operand2, true);
       return rexBuilder.makeCall(SqlStdOperatorTable.DATETIME_PLUS,
-          cx.convertExpression(call.operand(2)),
+          operand2b,
           multiply(rexBuilder,
               rexBuilder.makeIntervalLiteral(unit.multiplier,
                   new SqlIntervalQualifier(unit, null,
-                      unitLiteral.getParserPosition())),
-              cx.convertExpression(call.operand(1))));
+                      unitLiteral.getParserPosition())), operand1));
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/44f73389/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
index 7ab856b..f2109f4 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlOperatorBaseTest.java
@@ -84,6 +84,7 @@ import static org.hamcrest.CoreMatchers.equalTo;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertThat;
 import static org.junit.Assert.fail;
+import static org.junit.Assume.assumeTrue;
 
 /**
  * Contains unit tests for all operators. Each of the methods is named after an
@@ -6491,6 +6492,23 @@ public abstract class SqlOperatorBaseTest {
         "2016-02-29", "DATE NOT NULL");
   }
 
+  @Test public void testTimestampAddFractionalSeconds() {
+    tester.setFor(SqlStdOperatorTable.TIMESTAMP_ADD);
+    tester.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");
+
+    // The following test would correctly return "TIMESTAMP(6) NOT NULL" if max
+    // precision were 6 or higher
+    assumeTrue(tester.getValidator().getTypeFactory().getTypeSystem()
+        .getMaxPrecision(SqlTypeName.TIMESTAMP) == 3);
+    tester.checkType(
+        "timestampadd(MICROSECOND, 2, timestamp '2016-02-24 12:42:25.000000')",
+        // "2016-02-24 12:42:25.000002",
+        "TIMESTAMP(3) NOT NULL");
+  }
+
   @Test public void testTimestampDiff() {
     tester.setFor(SqlStdOperatorTable.TIMESTAMP_DIFF);
     tester.checkScalar("timestampdiff(HOUR, "