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/07/09 07:49:52 UTC

[30/30] calcite git commit: [CALCITE-2299] TIMESTAMPADD(SQL_TSI_FRAC_SECOND) should be nanoseconds (Sergey Nuyanzin)

[CALCITE-2299] TIMESTAMPADD(SQL_TSI_FRAC_SECOND) should be nanoseconds (Sergey Nuyanzin)

1) Add NANOSECONDS time unit;
2) Fix SQL_TSI_FRAC_SECOND which should be interpreted as nanoseconds;
3) Add tests.

Close apache/calcite#731


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

Branch: refs/heads/master
Commit: 3fa29455664bec0056c436491b369e0cd72242ea
Parents: df774b9
Author: snuyanzin <sn...@gmail.com>
Authored: Mon Jul 2 11:27:28 2018 +0300
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jul 8 22:46:20 2018 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  5 ++-
 .../calcite/sql/SqlIntervalQualifier.java       |  1 +
 .../sql/fun/SqlTimestampAddFunction.java        |  4 +--
 .../sql/fun/SqlTimestampDiffFunction.java       | 28 ++++++++++++---
 .../sql2rel/StandardConvertletTable.java        | 38 +++++++++++++-------
 .../calcite/sql/parser/SqlParserTest.java       |  4 +--
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 18 +++++++++-
 site/_docs/reference.md                         |  1 +
 8 files changed, 76 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/core/src/main/codegen/templates/Parser.jj
----------------------------------------------------------------------
diff --git a/core/src/main/codegen/templates/Parser.jj b/core/src/main/codegen/templates/Parser.jj
index a89352f..be05d9c 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -4107,7 +4107,8 @@ TimeUnit TimestampInterval() :
 {
     <FRAC_SECOND> { return TimeUnit.MICROSECOND; }
 |   <MICROSECOND> { return TimeUnit.MICROSECOND; }
-|   <SQL_TSI_FRAC_SECOND> { return TimeUnit.MICROSECOND; }
+|   <NANOSECOND> { return TimeUnit.NANOSECOND; }
+|   <SQL_TSI_FRAC_SECOND> { return TimeUnit.NANOSECOND; }
 |   <SQL_TSI_MICROSECOND> { return TimeUnit.MICROSECOND; }
 |   <SECOND> { return TimeUnit.SECOND; }
 |   <SQL_TSI_SECOND> { return TimeUnit.SECOND; }
@@ -5774,6 +5775,7 @@ SqlPostfixOperator PostfixRowOperator() :
 |   < MUMPS: "MUMPS" >
 |   < NAME: "NAME" >
 |   < NAMES: "NAMES" >
+|   < NANOSECOND: "NANOSECOND" >
 |   < NATIONAL: "NATIONAL" >
 |   < NATURAL: "NATURAL" >
 |   < NCHAR: "NCHAR" >
@@ -6261,6 +6263,7 @@ String CommonNonReservedKeyWord() :
     |   <MUMPS>
     |   <NAME>
     |   <NAMES>
+    |   <NANOSECOND>
     |   <NESTING>
     |   <NORMALIZED>
     |   <NULLABLE>

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index a25d93d..d5d9c74 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -166,6 +166,7 @@ public class SqlIntervalQualifier extends SqlNode {
     case MILLISECOND:
     case EPOCH:
     case MICROSECOND:
+    case NANOSECOND:
       return SqlTypeName.INTERVAL_SECOND;
     default:
       throw new AssertionError(timeUnitRange);

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/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 a9bf004..bc459d0 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
@@ -39,8 +39,8 @@ import org.apache.calcite.sql.type.SqlTypeName;
  * </blockquote>
  *
  * <p>The interval time unit can one of the following literals:<ul>
- * <li>MICROSECOND (and synonyms SQL_TSI_MICROSECOND, FRAC_SECOND,
- *     SQL_TSI_FRAC_SECOND)
+ * <li>NANOSECOND (and synonym SQL_TSI_FRAC_SECOND)
+ * <li>MICROSECOND (and synonyms SQL_TSI_MICROSECOND, FRAC_SECOND)
  * <li>SECOND (and synonym SQL_TSI_SECOND)
  * <li>MINUTE (and synonym  SQL_TSI_MINUTE)
  * <li>HOUR (and synonym  SQL_TSI_HOUR)

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
index 5ca25b3..d4aefa0 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
@@ -16,12 +16,17 @@
  */
 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.SqlFunction;
 import org.apache.calcite.sql.SqlFunctionCategory;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.type.OperandTypes;
-import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
 
 /**
  * The <code>TIMESTAMPDIFF</code> function, which calculates the difference
@@ -35,8 +40,8 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
  * </blockquote>
  *
  * <p>The interval time unit can one of the following literals:<ul>
- * <li>MICROSECOND (and synonyms SQL_TSI_MICROSECOND, FRAC_SECOND,
- *     SQL_TSI_FRAC_SECOND)
+ * <li>NANOSECOND (and synonym SQL_TSI_FRAC_SECOND)
+ * <li>MICROSECOND (and synonyms SQL_TSI_MICROSECOND, FRAC_SECOND)
  * <li>SECOND (and synonym SQL_TSI_SECOND)
  * <li>MINUTE (and synonym  SQL_TSI_MINUTE)
  * <li>HOUR (and synonym  SQL_TSI_HOUR)
@@ -52,9 +57,24 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
  */
 class SqlTimestampDiffFunction extends SqlFunction {
   /** Creates a SqlTimestampDiffFunction. */
+  private static final SqlReturnTypeInference RETURN_TYPE_INFERENCE =
+      new SqlReturnTypeInference() {
+        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          SqlTypeName sqlTypeName =
+              opBinding.getOperandLiteralValue(0, TimeUnit.class) == TimeUnit.NANOSECOND
+                  ? SqlTypeName.BIGINT
+                  : SqlTypeName.INTEGER;
+          return typeFactory.createTypeWithNullability(
+              typeFactory.createSqlType(sqlTypeName),
+              opBinding.getOperandType(1).isNullable()
+              || opBinding.getOperandType(2).isNullable());
+        }
+      };
+
   SqlTimestampDiffFunction() {
     super("TIMESTAMPDIFF", SqlKind.TIMESTAMP_DIFF,
-        ReturnTypes.INTEGER_NULLABLE, null,
+        RETURN_TYPE_INFERENCE, null,
         OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.DATETIME,
             SqlTypeFamily.DATETIME),
         SqlFunctionCategory.TIMEDATE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/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 5714806..e9b7cf6 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -60,7 +60,6 @@ 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;
@@ -1286,18 +1285,27 @@ 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);
+      RexNode interval2Add;
+      SqlIntervalQualifier qualifier =
+          new SqlIntervalQualifier(unit, null, unitLiteral.getParserPosition());
+      RexNode op1 = cx.convertExpression(call.operand(1));
+      switch (unit) {
+      case MICROSECOND:
+      case NANOSECOND:
+        interval2Add =
+            divide(rexBuilder,
+                multiply(rexBuilder,
+                    rexBuilder.makeIntervalLiteral(BigDecimal.ONE, qualifier), op1),
+                BigDecimal.ONE.divide(unit.multiplier,
+                    RoundingMode.UNNECESSARY));
+        break;
+      default:
+        interval2Add = multiply(rexBuilder,
+            rexBuilder.makeIntervalLiteral(unit.multiplier, qualifier), op1);
+      }
+
       return rexBuilder.makeCall(SqlStdOperatorTable.DATETIME_PLUS,
-          operand2b,
-          multiply(rexBuilder,
-              rexBuilder.makeIntervalLiteral(unit.multiplier,
-                  new SqlIntervalQualifier(unit, null,
-                      unitLiteral.getParserPosition())), operand1));
+          cx.convertExpression(call.operand(2)), interval2Add);
     }
   }
 
@@ -1311,9 +1319,13 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
       TimeUnit unit = unitLiteral.symbolValue(TimeUnit.class);
       BigDecimal multiplier = BigDecimal.ONE;
       BigDecimal divider = BigDecimal.ONE;
+      SqlTypeName sqlTypeName = unit == TimeUnit.NANOSECOND
+          ? SqlTypeName.BIGINT
+          : SqlTypeName.INTEGER;
       switch (unit) {
       case MICROSECOND:
       case MILLISECOND:
+      case NANOSECOND:
       case WEEK:
         multiplier = BigDecimal.valueOf(DateTimeUtils.MILLIS_PER_SECOND);
         divider = unit.multiplier;
@@ -1337,7 +1349,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
           ImmutableList.of(op2, op1));
       final RelDataType intType =
           cx.getTypeFactory().createTypeWithNullability(
-              cx.getTypeFactory().createSqlType(SqlTypeName.INTEGER),
+              cx.getTypeFactory().createSqlType(sqlTypeName),
               SqlTypeUtil.containsNullable(rexCall.getType()));
       RexNode e = rexBuilder.makeCast(intType, rexCall);
       return rexBuilder.multiplyDivide(e, multiplier, divider);

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
index 822c136..c3459f0 100644
--- a/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
+++ b/core/src/test/java/org/apache/calcite/sql/parser/SqlParserTest.java
@@ -6915,8 +6915,8 @@ public class SqlParserTest {
   @Test public void testTimestampAddAndDiff() {
     Map<String, List<String>> tsi = ImmutableMap.<String, List<String>>builder()
         .put("MICROSECOND",
-            Arrays.asList("FRAC_SECOND", "MICROSECOND",
-                "SQL_TSI_FRAC_SECOND", "SQL_TSI_MICROSECOND"))
+            Arrays.asList("FRAC_SECOND", "MICROSECOND", "SQL_TSI_MICROSECOND"))
+        .put("NANOSECOND", Arrays.asList("NANOSECOND", "SQL_TSI_FRAC_SECOND"))
         .put("SECOND", Arrays.asList("SECOND", "SQL_TSI_SECOND"))
         .put("MINUTE", Arrays.asList("MINUTE", "SQL_TSI_MINUTE"))
         .put("HOUR", Arrays.asList("HOUR", "SQL_TSI_HOUR"))

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/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 e8b163a..f2681f3 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
@@ -6486,10 +6486,22 @@ public abstract class SqlOperatorBaseTest {
   @Test public void testTimestampAdd() {
     tester.setFor(SqlStdOperatorTable.TIMESTAMP_ADD);
     tester.checkScalar(
+        "timestampadd(MICROSECOND, 2000000, timestamp '2016-02-24 12:42:25')",
+        "2016-02-24 12:42:27",
+        "TIMESTAMP(3) NOT NULL");
+    tester.checkScalar(
         "timestampadd(SQL_TSI_SECOND, 2, timestamp '2016-02-24 12:42:25')",
         "2016-02-24 12:42:27",
         "TIMESTAMP(0) NOT NULL");
     tester.checkScalar(
+        "timestampadd(NANOSECOND, 3000000000, timestamp '2016-02-24 12:42:25')",
+        "2016-02-24 12:42:28",
+        "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar(
+        "timestampadd(SQL_TSI_FRAC_SECOND, 2000000000, timestamp '2016-02-24 12:42:25')",
+        "2016-02-24 12:42:27",
+        "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar(
         "timestampadd(MINUTE, 2, timestamp '2016-02-24 12:42:25')",
         "2016-02-24 12:44:25",
         "TIMESTAMP(0) NOT NULL");
@@ -6565,7 +6577,11 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalar("timestampdiff(SQL_TSI_FRAC_SECOND, "
         + "timestamp '2016-02-24 12:42:25', "
         + "timestamp '2016-02-24 12:42:20')",
-        "-5000000", "INTEGER NOT NULL");
+        "-5000000000", "BIGINT NOT NULL");
+    tester.checkScalar("timestampdiff(NANOSECOND, "
+        + "timestamp '2016-02-24 12:42:25', "
+        + "timestamp '2016-02-24 12:42:20')",
+        "-5000000000", "BIGINT NOT NULL");
     tester.checkScalar("timestampdiff(YEAR, "
         + "timestamp '2014-02-24 12:42:25', "
         + "timestamp '2016-02-24 12:42:25')",

http://git-wip-us.apache.org/repos/asf/calcite/blob/3fa29455/site/_docs/reference.md
----------------------------------------------------------------------
diff --git a/site/_docs/reference.md b/site/_docs/reference.md
index 3d1433f..b8bde09 100644
--- a/site/_docs/reference.md
+++ b/site/_docs/reference.md
@@ -614,6 +614,7 @@ MORE,
 MUMPS,
 NAME,
 NAMES,
+NANOSECOND,
 **NATIONAL**,
 **NATURAL**,
 **NCHAR**,