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 2016/07/13 02:55:06 UTC

calcite git commit: [CALCITE-1312] Return type of TIMESTAMP_ADD applied to a DATE should be TIMESTAMP if unit is smaller than DAY (Minji Kim)

Repository: calcite
Updated Branches:
  refs/heads/master b4f28d7e2 -> db6a3b3b2


[CALCITE-1312] Return type of TIMESTAMP_ADD applied to a DATE should be TIMESTAMP if unit is smaller than DAY (Minji Kim)

Handle null values in TIMESTAMP_DIFF(date, date).

Close apache/calcite#253


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

Branch: refs/heads/master
Commit: db6a3b3b298df81a199aff6003a32b67d74ba643
Parents: b4f28d7
Author: Minji Kim <mi...@dremio.com>
Authored: Thu Jun 16 12:04:37 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Jul 12 19:54:35 2016 -0700

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java | 45 +++++----
 .../apache/calcite/runtime/SqlFunctions.java    |  4 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java |  5 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    | 97 +++++++-------------
 .../sql/fun/SqlTimestampAddFunction.java        | 89 ++++++++++++++++++
 .../sql/fun/SqlTimestampDiffFunction.java       | 64 +++++++++++++
 .../sql2rel/StandardConvertletTable.java        | 21 +++--
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 55 +++++++++++
 .../apache/calcite/test/SqlFunctionsTest.java   |  6 +-
 9 files changed, 293 insertions(+), 93 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexImpTable.java
----------------------------------------------------------------------
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 22f2249..38466e1 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
@@ -1954,28 +1954,37 @@ public class RexImpTable {
     public Expression implement(RexToLixTranslator translator, RexCall call,
         List<Expression> translatedOperands) {
       final RexNode operand0 = call.getOperands().get(0);
-      final Expression trop0 = translatedOperands.get(0);
+      Expression trop0 = translatedOperands.get(0);
       final SqlTypeName typeName1 =
           call.getOperands().get(1).getType().getSqlTypeName();
       Expression trop1 = translatedOperands.get(1);
+      final SqlTypeName typeName = call.getType().getSqlTypeName();
       switch (operand0.getType().getSqlTypeName()) {
       case DATE:
-        switch (typeName1) {
-        case INTERVAL_DAY:
-        case INTERVAL_DAY_HOUR:
-        case INTERVAL_DAY_MINUTE:
-        case INTERVAL_DAY_SECOND:
-        case INTERVAL_HOUR:
-        case INTERVAL_HOUR_MINUTE:
-        case INTERVAL_HOUR_SECOND:
-        case INTERVAL_MINUTE:
-        case INTERVAL_MINUTE_SECOND:
-        case INTERVAL_SECOND:
-          trop1 =
-              Expressions.convert_(
-                  Expressions.divide(trop1,
-                      Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
-                  int.class);
+        switch (typeName) {
+        case TIMESTAMP:
+          trop0 = Expressions.convert_(
+              Expressions.multiply(trop0,
+                  Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+              long.class);
+          break;
+        default:
+          switch (typeName1) {
+          case INTERVAL_DAY:
+          case INTERVAL_DAY_HOUR:
+          case INTERVAL_DAY_MINUTE:
+          case INTERVAL_DAY_SECOND:
+          case INTERVAL_HOUR:
+          case INTERVAL_HOUR_MINUTE:
+          case INTERVAL_HOUR_SECOND:
+          case INTERVAL_MINUTE:
+          case INTERVAL_MINUTE_SECOND:
+          case INTERVAL_SECOND:
+            trop1 = Expressions.convert_(
+                Expressions.divide(trop1,
+                    Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                int.class);
+          }
         }
         break;
       case TIME:
@@ -2012,7 +2021,7 @@ public class RexImpTable {
       default:
         switch (call.getKind()) {
         case MINUS:
-          switch (call.getType().getSqlTypeName()) {
+          switch (typeName) {
           case INTERVAL_YEAR:
           case INTERVAL_YEAR_MONTH:
           case INTERVAL_MONTH:

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
----------------------------------------------------------------------
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 88c9356..0794a1a 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -1650,7 +1650,7 @@ public class SqlFunctions {
 
   /** Finds the number of months between two dates, each represented as the
    * number of days since the epoch. */
-  public static long subtractMonths(int date0, int date1) {
+  public static int subtractMonths(int date0, int date1) {
     if (date0 < date1) {
       return -subtractMonths(date1, date0);
     }
@@ -1679,7 +1679,7 @@ public class SqlFunctions {
         DateTimeUtils.floorMod(t1, DateTimeUtils.MILLIS_PER_DAY);
     final int d1 = (int) DateTimeUtils.floorDiv(t1 - millis1,
         DateTimeUtils.MILLIS_PER_DAY);
-    int x = (int) subtractMonths(d0, d1);
+    int x = subtractMonths(d0, d1);
     final long d2 = addMonths(d1, x);
     if (d2 == d0 && millis0 < millis1) {
       --x;

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 95b1bc3..8121e20 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -304,7 +304,10 @@ public class SqlLiteral extends SqlNode {
    */
   public static Comparable value(SqlNode node) {
     if (node instanceof SqlLiteral) {
-      SqlLiteral literal = (SqlLiteral) node;
+      final SqlLiteral literal = (SqlLiteral) node;
+      if (literal.getTypeName() == SqlTypeName.SYMBOL) {
+        return (Enum) literal.value;
+      }
       switch (literal.getTypeName().getFamily()) {
       case CHARACTER:
         return (NlsString) literal.value;

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
index b6f4ed3..85f576a 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlStdOperatorTable.java
@@ -16,6 +16,9 @@
  */
 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.SqlAggFunction;
 import org.apache.calcite.sql.SqlAsOperator;
 import org.apache.calcite.sql.SqlBinaryOperator;
@@ -29,6 +32,7 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlLiteral;
 import org.apache.calcite.sql.SqlOperandCountRange;
 import org.apache.calcite.sql.SqlOperator;
+import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlOverOperator;
 import org.apache.calcite.sql.SqlPostfixOperator;
 import org.apache.calcite.sql.SqlPrefixOperator;
@@ -43,10 +47,11 @@ import org.apache.calcite.sql.SqlValuesOperator;
 import org.apache.calcite.sql.SqlWindow;
 import org.apache.calcite.sql.SqlWriter;
 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.SqlOperandCountRanges;
-import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ReflectiveSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlModality;
@@ -454,7 +459,31 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlKind.PLUS,
           40,
           true,
-          ReturnTypes.ARG0_NULLABLE,
+          new SqlReturnTypeInference() {
+        @Override public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          final RelDataType unit = opBinding.getOperandType(1);
+          final TimeUnit addUnit;
+          if (unit instanceof IntervalSqlType) {
+            addUnit = unit.getIntervalQualifier().getStartUnit();
+          } else {
+            addUnit = null;
+          }
+          switch (addUnit) {
+          case HOUR:
+          case MINUTE:
+          case SECOND:
+          case MILLISECOND:
+          case MICROSECOND:
+            return typeFactory.createTypeWithNullability(
+                typeFactory.createSqlType(SqlTypeName.TIMESTAMP),
+                opBinding.getOperandType(0).isNullable()
+                    || opBinding.getOperandType(1).isNullable());
+          default:
+            return opBinding.getOperandType(0);
+          }
+        }
+      },
           InferTypes.FIRST_KNOWN,
           OperandTypes.PLUS_OPERATOR);
 
@@ -1321,67 +1350,11 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
   public static final SqlFunction CURRENT_DATE =
       new SqlCurrentDateFunction();
 
-  /**
-   * <p>The <code>TIMESTAMPADD</code> function, which adds an interval to a
-   * timestamp.
-   *
-   * <p>The SQL syntax is
-   *
-   * <blockquote>
-   * <code>TIMESTAMPADD(<i>timestamp interval</i>, <i>quantity</i>, <i>timestamp</i>)</code>
-   * </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>SECOND (and synonym SQL_TSI_SECOND)
-   * <li>MINUTE (and synonym  SQL_TSI_MINUTE)
-   * <li>HOUR (and synonym  SQL_TSI_HOUR)
-   * <li>DAY (and synonym SQL_TSI_DAY)
-   * <li>WEEK (and synonym  SQL_TSI_WEEK)
-   * <li>MONTH (and synonym SQL_TSI_MONTH)
-   * <li>QUARTER (and synonym SQL_TSI_QUARTER)
-   * <li>YEAR (and synonym  SQL_TSI_YEAR)
-   * </ul>
-   *
-   * <p>Returns modified timestamp.
-   */
-  public static final SqlFunction TIMESTAMP_ADD =
-      new SqlFunction("TIMESTAMPADD", SqlKind.TIMESTAMP_ADD,
-          ReturnTypes.ARG2_NULLABLE, null,
-          OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.INTEGER,
-              SqlTypeFamily.DATETIME), SqlFunctionCategory.TIMEDATE);
+  /** The <code>TIMESTAMPADD</code> function. */
+  public static final SqlFunction TIMESTAMP_ADD = new SqlTimestampAddFunction();
 
-  /**
-   * <p>The <code>TIMESTAMPDIFF</code> function, which calculates the difference
-   * between two timestamps.
-   *
-   * <p>The SQL syntax is
-   *
-   * <blockquote>
-   * <code>TIMESTAMPDIFF(<i>timestamp interval</i>, <i>timestamp</i>, <i>timestamp</i>)</code>
-   * </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>SECOND (and synonym SQL_TSI_SECOND)
-   * <li>MINUTE (and synonym  SQL_TSI_MINUTE)
-   * <li>HOUR (and synonym  SQL_TSI_HOUR)
-   * <li>DAY (and synonym SQL_TSI_DAY)
-   * <li>WEEK (and synonym  SQL_TSI_WEEK)
-   * <li>MONTH (and synonym SQL_TSI_MONTH)
-   * <li>QUARTER (and synonym SQL_TSI_QUARTER)
-   * <li>YEAR (and synonym  SQL_TSI_YEAR)
-   * </ul>
-   *
-   * <p>Returns difference between two timestamps in indicated timestamp interval.
-   */
-  public static final SqlFunction TIMESTAMP_DIFF =
-      new SqlFunction("TIMESTAMPDIFF", SqlKind.TIMESTAMP_DIFF,
-          ReturnTypes.INTEGER_NULLABLE, null,
-          OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.DATETIME,
-              SqlTypeFamily.DATETIME), SqlFunctionCategory.TIMEDATE);
+  /** The <code>TIMESTAMPDIFF</code> function. */
+  public static final SqlFunction TIMESTAMP_DIFF = new SqlTimestampDiffFunction();
 
   /**
    * Use of the <code>IN_FENNEL</code> operator forces the argument to be

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/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
new file mode 100644
index 0000000..b71bfec
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampAddFunction.java
@@ -0,0 +1,89 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+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.SqlReturnTypeInference;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+import org.apache.calcite.sql.type.SqlTypeName;
+
+/**
+ * The <code>TIMESTAMPADD</code> function, which adds an interval to a
+ * timestamp.
+ *
+ * <p>The SQL syntax is
+ *
+ * <blockquote>
+ * <code>TIMESTAMPADD(<i>timestamp interval</i>, <i>quantity</i>,
+ * <i>timestamp</i>)</code>
+ * </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>SECOND (and synonym SQL_TSI_SECOND)
+ * <li>MINUTE (and synonym  SQL_TSI_MINUTE)
+ * <li>HOUR (and synonym  SQL_TSI_HOUR)
+ * <li>DAY (and synonym SQL_TSI_DAY)
+ * <li>WEEK (and synonym  SQL_TSI_WEEK)
+ * <li>MONTH (and synonym SQL_TSI_MONTH)
+ * <li>QUARTER (and synonym SQL_TSI_QUARTER)
+ * <li>YEAR (and synonym  SQL_TSI_YEAR)
+ * </ul>
+ *
+ * <p>Returns modified timestamp.
+ */
+class SqlTimestampAddFunction extends SqlFunction {
+
+  private static final SqlReturnTypeInference RETURN_TYPE_INFERENCE =
+      new SqlReturnTypeInference() {
+        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+          final RelDataTypeFactory typeFactory = opBinding.getTypeFactory();
+          final TimeUnit unit = (TimeUnit) opBinding.getOperandLiteralValue(0);
+          switch (unit) {
+          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);
+          }
+        }
+      };
+
+  /** Creates a SqlTimestampAddFunction. */
+  SqlTimestampAddFunction() {
+    super("TIMESTAMPADD", SqlKind.TIMESTAMP_ADD, RETURN_TYPE_INFERENCE, null,
+        OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.INTEGER,
+            SqlTypeFamily.DATETIME),
+        SqlFunctionCategory.TIMEDATE);
+  }
+}
+
+// End SqlTimestampAddFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/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
new file mode 100644
index 0000000..5ca25b3
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlTimestampDiffFunction.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to you under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.calcite.sql.fun;
+
+import org.apache.calcite.sql.SqlFunction;
+import org.apache.calcite.sql.SqlFunctionCategory;
+import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.sql.type.OperandTypes;
+import org.apache.calcite.sql.type.ReturnTypes;
+import org.apache.calcite.sql.type.SqlTypeFamily;
+
+/**
+ * The <code>TIMESTAMPDIFF</code> function, which calculates the difference
+ * between two timestamps.
+ *
+ * <p>The SQL syntax is
+ *
+ * <blockquote>
+ * <code>TIMESTAMPDIFF(<i>timestamp interval</i>, <i>timestamp</i>,
+ * <i>timestamp</i>)</code>
+ * </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>SECOND (and synonym SQL_TSI_SECOND)
+ * <li>MINUTE (and synonym  SQL_TSI_MINUTE)
+ * <li>HOUR (and synonym  SQL_TSI_HOUR)
+ * <li>DAY (and synonym SQL_TSI_DAY)
+ * <li>WEEK (and synonym  SQL_TSI_WEEK)
+ * <li>MONTH (and synonym SQL_TSI_MONTH)
+ * <li>QUARTER (and synonym SQL_TSI_QUARTER)
+ * <li>YEAR (and synonym  SQL_TSI_YEAR)
+ * </ul>
+ *
+ * <p>Returns difference between two timestamps in indicated timestamp
+ * interval.
+ */
+class SqlTimestampDiffFunction extends SqlFunction {
+  /** Creates a SqlTimestampDiffFunction. */
+  SqlTimestampDiffFunction() {
+    super("TIMESTAMPDIFF", SqlKind.TIMESTAMP_DIFF,
+        ReturnTypes.INTEGER_NULLABLE, null,
+        OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.DATETIME,
+            SqlTypeFamily.DATETIME),
+        SqlFunctionCategory.TIMEDATE);
+  }
+}
+
+// End SqlTimestampDiffFunction.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/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 21aaf30..fdac4d9 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -69,6 +69,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -1530,16 +1531,22 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
         unit = TimeUnit.MONTH;
         break;
       }
-      final RelDataType intType =
-          cx.getTypeFactory().createSqlType(SqlTypeName.INTEGER);
       final SqlIntervalQualifier qualifier =
           new SqlIntervalQualifier(unit, null, SqlParserPos.ZERO);
+      final RexNode op2 = cx.convertExpression(call.operand(2));
+      final RexNode op1 = cx.convertExpression(call.operand(1));
       final RelDataType intervalType =
-          cx.getTypeFactory().createSqlIntervalType(qualifier);
-      RexNode e = rexBuilder.makeCast(intType,
-          rexBuilder.makeCall(intervalType, SqlStdOperatorTable.MINUS_DATE,
-              ImmutableList.of(cx.convertExpression(call.operand(2)),
-                  cx.convertExpression(call.operand(1)))));
+          cx.getTypeFactory().createTypeWithNullability(
+              cx.getTypeFactory().createSqlIntervalType(qualifier),
+              op1.getType().isNullable() || op2.getType().isNullable());
+      final RexCall rexCall = (RexCall) rexBuilder.makeCall(
+          intervalType, SqlStdOperatorTable.MINUS_DATE,
+          ImmutableList.of(op2, op1));
+      final RelDataType intType =
+          cx.getTypeFactory().createTypeWithNullability(
+              cx.getTypeFactory().createSqlType(SqlTypeName.INTEGER),
+              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/db6a3b3b/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 fe51c64..4442cc5 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
@@ -5056,6 +5056,27 @@ public abstract class SqlOperatorBaseTest {
     tester.checkScalar(
         "timestampadd(MONTH, 3, timestamp '2016-02-24 12:42:25')",
         "2016-05-24 12:42:25", "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar(
+        "timestampadd(MONTH, 3, cast(null as timestamp))",
+        null, "TIMESTAMP(0)");
+
+    // TIMESTAMPADD with DATE; returns a TIMESTAMP value for sub-day intervals.
+    tester.checkScalar("timestampadd(MONTH, 1, date '2016-06-15')",
+        "2016-07-15", "DATE NOT NULL");
+    tester.checkScalar("timestampadd(DAY, 1, date '2016-06-15')",
+        "2016-06-16", "DATE NOT NULL");
+    tester.checkScalar("timestampadd(HOUR, -1, date '2016-06-15')",
+        "2016-06-14 23:00:00", "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar("timestampadd(MINUTE, 1, date '2016-06-15')",
+        "2016-06-15 00:01:00", "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar("timestampadd(SQL_TSI_SECOND, -1, date '2016-06-15')",
+        "2016-06-14 23:59:59", "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar("timestampadd(SECOND, 1, date '2016-06-15')",
+        "2016-06-15 00:00:01", "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar("timestampadd(SECOND, 1, cast(null as date))",
+        null, "TIMESTAMP(0)");
+    tester.checkScalar("timestampadd(DAY, 1, cast(null as date))",
+        null, "DATE");
   }
 
   @Test public void testTimestampDiff() {
@@ -5096,6 +5117,40 @@ public abstract class SqlOperatorBaseTest {
         + "timestamp '2014-02-24 12:42:25', "
         + "timestamp '2614-02-24 12:42:25')",
         "(?s)Encountered \"CENTURY\" at .*", false);
+    tester.checkScalar("timestampdiff(QUARTER, "
+        + "timestamp '2014-02-24 12:42:25', "
+        + "cast(null as timestamp))",
+        null, "INTEGER");
+    tester.checkScalar("timestampdiff(QUARTER, "
+        + "cast(null as timestamp), "
+        + "timestamp '2014-02-24 12:42:25')",
+        null, "INTEGER");
+
+    // timestampdiff with date
+    tester.checkScalar(
+        "timestampdiff(MONTH, date '2016-03-15', date '2016-06-14')",
+        "2",
+        "INTEGER NOT NULL");
+    tester.checkScalar(
+        "timestampdiff(DAY, date '2016-06-15', date '2016-06-14')",
+        "-1",
+        "INTEGER NOT NULL");
+    tester.checkScalar(
+        "timestampdiff(HOUR, date '2016-06-15', date '2016-06-14')",
+        "-24",
+        "INTEGER NOT NULL");
+    tester.checkScalar(
+        "timestampdiff(MINUTE, date '2016-06-15',  date '2016-06-15')",
+        "0",
+        "INTEGER NOT NULL");
+    tester.checkScalar(
+        "timestampdiff(SECOND, cast(null as date), date '2016-06-15')",
+        null,
+        "INTEGER");
+    tester.checkScalar(
+        "timestampdiff(DAY, date '2016-06-15', cast(null as date))",
+        null,
+        "INTEGER");
   }
 
   @Test public void testDenseRankFunc() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/db6a3b3b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
----------------------------------------------------------------------
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 241ecb7..e6b6d44 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -381,11 +381,11 @@ public class SqlFunctionsTest {
     assertThat((int) date, is(date1));
 
     assertThat(subtractMonths(date1, date0),
-        anyOf(is((long) months), is((long) months + 1)));
+        anyOf(is(months), is(months + 1)));
     assertThat(subtractMonths(date1 + 1, date0),
-        anyOf(is((long) months), is((long) months + 1)));
+        anyOf(is(months), is(months + 1)));
     assertThat(subtractMonths(date1, date0 + 1),
-        anyOf(is((long) months), is((long) months - 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)),