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/06/27 23:42:19 UTC

calcite git commit: [CALCITE-308] DATE + INTERVAL gives wrong result [Forced Update!]

Repository: calcite
Updated Branches:
  refs/heads/master 29f518f0d -> 5511253b5 (forced update)


[CALCITE-308] DATE + INTERVAL gives wrong result

All tests for operations on intervals now succeed.


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

Branch: refs/heads/master
Commit: 5511253b5c46741dfed8e1ac8d4fa23025aeb3c5
Parents: a445ff2
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jun 13 09:58:22 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Mon Jun 27 14:14:19 2016 -0700

----------------------------------------------------------------------
 .../calcite/adapter/enumerable/RexImpTable.java |  91 ++++++++++++--
 .../adapter/enumerable/RexToLixTranslator.java  |   9 ++
 .../java/org/apache/calcite/rex/RexBuilder.java | 123 +++++++++----------
 .../java/org/apache/calcite/rex/RexLiteral.java |   3 +
 .../rex/RexSqlStandardConvertletTable.java      |  38 +++---
 .../apache/calcite/runtime/SqlFunctions.java    |  98 +++++++++++++++
 .../calcite/sql/SqlIntervalQualifier.java       |   6 +
 .../sql/fun/SqlDatetimeSubtractionOperator.java |   6 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |  14 +--
 .../calcite/sql/type/IntervalSqlType.java       |  17 ++-
 .../apache/calcite/sql/type/ReturnTypes.java    |  43 ++++---
 .../sql2rel/StandardConvertletTable.java        |  91 +++++++++-----
 .../org/apache/calcite/util/BuiltInMethod.java  |   3 +
 .../calcite/sql/test/SqlOperatorBaseTest.java   | 115 ++++++++---------
 .../java/org/apache/calcite/test/JdbcTest.java  |  13 +-
 .../apache/calcite/test/SqlFunctionsTest.java   |  43 +++++++
 .../apache/calcite/test/SqlValidatorTest.java   |   2 +-
 17 files changed, 485 insertions(+), 230 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 a582a86..2ac0dc9 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
@@ -45,6 +45,7 @@ import org.apache.calcite.schema.ImplementableFunction;
 import org.apache.calcite.schema.impl.AggregateFunctionImpl;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlBinaryOperator;
+import org.apache.calcite.sql.SqlIntervalQualifier;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.fun.SqlTrimFunction;
@@ -946,6 +947,34 @@ public class RexImpTable {
     return Expressions.constant(null, type);
   }
 
+  /** Multiplies an expression by a constant and divides by another constant,
+   * optimizing appropriately.
+   *
+   * <p>For example, {@code multiplyDivide(e, 10, 1000)} returns
+   * {@code e / 100}. */
+  public static Expression multiplyDivide(Expression e, BigDecimal multiplier,
+      BigDecimal divider) {
+    if (multiplier.equals(BigDecimal.ONE)) {
+      if (divider.equals(BigDecimal.ONE)) {
+        return e;
+      }
+      return Expressions.divide(e,
+          Expressions.constant(divider.intValueExact()));
+    }
+    final BigDecimal x =
+        multiplier.divide(divider, BigDecimal.ROUND_UNNECESSARY);
+    switch (x.compareTo(BigDecimal.ONE)) {
+    case 0:
+      return e;
+    case 1:
+      return Expressions.multiply(e, Expressions.constant(x.intValueExact()));
+    case -1:
+      return multiplyDivide(e, BigDecimal.ONE, x);
+    default:
+      throw new AssertionError();
+    }
+  }
+
   /** Implementor for the {@code COUNT} aggregate function. */
   static class CountImplementor extends StrictAggImplementor {
     @Override public void implementNotNullAdd(AggContext info,
@@ -1635,6 +1664,7 @@ public class RexImpTable {
             Primitive.ofBoxOr(expressions.get(0).getType());
         final SqlBinaryOperator op = (SqlBinaryOperator) call.getOperator();
         if (primitive == null
+            || expressions.get(1).getType() == BigDecimal.class
             || COMPARISON_OPERATORS.contains(op)
             && !COMP_OP_TYPES.contains(primitive)) {
           return Expressions.call(
@@ -1643,8 +1673,12 @@ public class RexImpTable {
               expressions);
         }
       }
-      return Expressions.makeBinary(
-          expressionType, expressions.get(0), expressions.get(1));
+
+      final Type returnType =
+          translator.typeFactory.getJavaClass(call.getType());
+      return Types.castIfNecessary(returnType,
+          Expressions.makeBinary(expressionType, expressions.get(0),
+              expressions.get(1)));
     }
   }
 
@@ -1922,24 +1956,59 @@ public class RexImpTable {
         List<Expression> translatedOperands) {
       final RexNode operand0 = call.getOperands().get(0);
       final Expression trop0 = translatedOperands.get(0);
+      final SqlTypeName typeName1 =
+          call.getOperands().get(1).getType().getSqlTypeName();
       Expression trop1 = translatedOperands.get(1);
       switch (operand0.getType().getSqlTypeName()) {
       case DATE:
-        trop1 =
-            Expressions.convert_(
-                Expressions.divide(trop1,
-                    Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
-                int.class);
+        switch (typeName1) {
+        case INTERVAL_DAY_TIME:
+          trop1 =
+              Expressions.convert_(
+                  Expressions.divide(trop1,
+                      Expressions.constant(DateTimeUtils.MILLIS_PER_DAY)),
+                  int.class);
+        }
         break;
       case TIME:
         trop1 = Expressions.convert_(trop1, int.class);
         break;
       }
-      switch (call.getKind()) {
-      case MINUS:
-        return Expressions.subtract(trop0, trop1);
+      final SqlIntervalQualifier interval = call.getType().getIntervalQualifier();
+      switch (typeName1) {
+      case INTERVAL_YEAR_MONTH:
+        switch (call.getKind()) {
+        case MINUS:
+          trop1 = Expressions.negate(trop1);
+        }
+        return Expressions.call(BuiltInMethod.ADD_MONTHS.method, trop0, trop1);
+
+      case INTERVAL_DAY_TIME:
+        switch (call.getKind()) {
+        case MINUS:
+          return Expressions.subtract(trop0, trop1);
+        default:
+          return Expressions.add(trop0, trop1);
+        }
+
       default:
-        return Expressions.add(trop0, trop1);
+        switch (call.getKind()) {
+        case MINUS:
+          Class targetType = interval.isYearMonth() ? int.class : long.class;
+          if (interval.isYearMonth()) {
+            return Expressions.call(BuiltInMethod.SUBTRACT_MONTHS.method,
+                trop0, trop1);
+          }
+          TimeUnit fromUnit =
+              typeName1 == SqlTypeName.DATE ? TimeUnit.DAY : TimeUnit.MILLISECOND;
+          TimeUnit toUnit = interval.isYearMonth() ? TimeUnit.MONTH : TimeUnit.MILLISECOND;
+          return multiplyDivide(
+              Expressions.convert_(Expressions.subtract(trop0, trop1),
+                  targetType),
+              fromUnit.multiplier, toUnit.multiplier);
+        default:
+          return Expressions.add(trop0, trop1);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
index baba915..2a93c0a 100644
--- a/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
+++ b/core/src/main/java/org/apache/calcite/adapter/enumerable/RexToLixTranslator.java
@@ -413,6 +413,15 @@ public class RexToLixTranslator {
                     (long) Math.pow(10, 3 - targetScale)));
       }
       break;
+    case INTERVAL_DAY_TIME:
+    case INTERVAL_YEAR_MONTH:
+      switch (sourceType.getSqlTypeName().getFamily()) {
+      case NUMERIC:
+        final SqlIntervalQualifier interval = targetType.getIntervalQualifier();
+        final BigDecimal multiplier = interval.getUnit().multiplier;
+        final BigDecimal divider = BigDecimal.ONE;
+        convert = RexImpTable.multiplyDivide(convert, multiplier, divider);
+      }
     }
     return convert;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
index 2d8b459..825fdb9 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -511,17 +511,29 @@ public class RexBuilder {
           break;
         case INTERVAL_DAY_TIME:
           assert value instanceof BigDecimal;
-          BigDecimal value2 = (BigDecimal) value;
-          final BigDecimal multiplier =
-              literal.getType().getIntervalQualifier().getStartUnit()
-                  .multiplier;
           typeName = type.getSqlTypeName();
+          switch (typeName) {
+          case BIGINT:
+          case INTEGER:
+          case SMALLINT:
+          case TINYINT:
+          case FLOAT:
+          case REAL:
+          case DECIMAL:
+            BigDecimal value2 = (BigDecimal) value;
+            final BigDecimal multiplier =
+                baseUnit(literal.getType().getIntervalQualifier().getUnit()).multiplier;
+            final BigDecimal divider =
+                literal.getType().getIntervalQualifier().getUnit().multiplier;
+            value = value2.multiply(multiplier)
+                .divide(divider, 0, BigDecimal.ROUND_HALF_DOWN);
+          }
+
           // Not all types are allowed for literals
           switch (typeName) {
           case INTEGER:
             typeName = SqlTypeName.BIGINT;
           }
-          value = value2.divide(multiplier, 0, BigDecimal.ROUND_HALF_DOWN);
         }
         final RexLiteral literal2 =
             makeLiteral(value, type, typeName);
@@ -532,9 +544,6 @@ public class RexBuilder {
         }
         return literal2;
       }
-    } else if (SqlTypeUtil.isInterval(type)
-        && SqlTypeUtil.isExactNumeric(exp.getType())) {
-      return makeCastExactToInterval(type, exp);
     } else if (SqlTypeUtil.isExactNumeric(type)
         && SqlTypeUtil.isInterval(exp.getType())) {
       return makeCastIntervalToExact(type, exp);
@@ -548,6 +557,17 @@ public class RexBuilder {
     return makeAbstractCast(type, exp);
   }
 
+  /** Returns the lowest granularity unit for the given unit.
+   * YEAR and MONTH intervals are stored as months;
+   * HOUR, MINUTE, SECOND intervals are stored as milliseconds. */
+  protected static TimeUnit baseUnit(TimeUnit unit) {
+    if (unit.yearMonth) {
+      return TimeUnit.MONTH;
+    } else {
+      return TimeUnit.MILLISECOND;
+    }
+  }
+
   private boolean canRemoveCastFromLiteral(RelDataType toType, Comparable value,
       SqlTypeName fromTypeName) {
     final SqlTypeName sqlType = toType.getSqlTypeName();
@@ -604,25 +624,14 @@ public class RexBuilder {
   }
 
   private RexNode makeCastIntervalToExact(RelDataType toType, RexNode exp) {
-    IntervalSqlType intervalType = (IntervalSqlType) exp.getType();
-    TimeUnit endUnit = intervalType.getIntervalQualifier().getEndUnit();
-    if (endUnit == null) {
-      endUnit = intervalType.getIntervalQualifier().getStartUnit();
-    }
-    int scale = 0;
-    if (endUnit == TimeUnit.SECOND) {
-      scale = Math.min(
-          intervalType.getIntervalQualifier()
-              .getFractionalSecondPrecision(typeFactory.getTypeSystem()),
-          3);
-    }
-    BigDecimal multiplier = endUnit.multiplier.scaleByPowerOfTen(-scale);
-    RexNode value = decodeIntervalOrDecimal(exp);
-    if (multiplier.longValue() != 1) {
-      value = makeCall(
-          SqlStdOperatorTable.DIVIDE_INTEGER,
-          value, makeBigintLiteral(multiplier));
-    }
+    final IntervalSqlType intervalType = (IntervalSqlType) exp.getType();
+    final TimeUnit endUnit = intervalType.getIntervalQualifier().getUnit();
+    final TimeUnit baseUnit = baseUnit(endUnit);
+    final BigDecimal multiplier = baseUnit.multiplier;
+    final int scale = 0;
+    BigDecimal divider = endUnit.multiplier.scaleByPowerOfTen(-scale);
+    RexNode value = multiplyDivide(decodeIntervalOrDecimal(exp),
+        multiplier, divider);
     if (scale > 0) {
       RelDataType decimalType =
           getTypeFactory().createSqlType(
@@ -634,30 +643,26 @@ public class RexBuilder {
     return ensureType(toType, value, false);
   }
 
-  private RexNode makeCastExactToInterval(RelDataType toType, RexNode exp) {
-    IntervalSqlType intervalType = (IntervalSqlType) toType;
-    TimeUnit endUnit = intervalType.getIntervalQualifier().getEndUnit();
-    if (endUnit == null) {
-      endUnit = intervalType.getIntervalQualifier().getStartUnit();
-    }
-    int scale = 0;
-    if (endUnit == TimeUnit.SECOND) {
-      scale = Math.min(
-          intervalType.getIntervalQualifier()
-              .getFractionalSecondPrecision(typeFactory.getTypeSystem()),
-          3);
-    }
-    BigDecimal multiplier = endUnit.multiplier.scaleByPowerOfTen(-scale);
-    RelDataType decimalType =
-        getTypeFactory().createSqlType(SqlTypeName.DECIMAL,
-            scale + intervalType.getPrecision(),
-            scale);
-    RexNode value = decodeIntervalOrDecimal(ensureType(decimalType, exp, true));
-    if (multiplier.longValue() != 1) {
-      value = makeCall(SqlStdOperatorTable.MULTIPLY,
-          value, makeExactLiteral(multiplier));
+  private RexNode multiplyDivide(RexNode e, BigDecimal multiplier,
+      BigDecimal divider) {
+    assert multiplier.signum() > 0;
+    assert divider.signum() > 0;
+    switch (multiplier.compareTo(divider)) {
+    case 0:
+      return e;
+    case 1:
+      // E.g. multiplyDivide(e, 1000, 10) ==> e * 100
+      return makeCall(SqlStdOperatorTable.MULTIPLY, e,
+          makeExactLiteral(
+              multiplier.divide(divider, BigDecimal.ROUND_UNNECESSARY)));
+    case -1:
+      // E.g. multiplyDivide(e, 10, 1000) ==> e / 100
+      return makeCall(SqlStdOperatorTable.DIVIDE_INTEGER, e,
+          makeExactLiteral(
+              divider.divide(multiplier, BigDecimal.ROUND_UNNECESSARY)));
+    default:
+      throw new AssertionError(multiplier + "/" + divider);
     }
-    return encodeIntervalOrDecimal(value, toType, false);
   }
 
   /**
@@ -741,24 +746,6 @@ public class RexBuilder {
   }
 
   /**
-   * Makes an expression which converts a value of type T to a value of type T
-   * NOT NULL, or throws if the value is NULL. If the expression is already
-   * NOT NULL, does nothing.
-   */
-  public RexNode makeNotNullCast(RexNode expr) {
-    RelDataType type = expr.getType();
-    if (!type.isNullable()) {
-      return expr;
-    }
-    RelDataType typeNotNull =
-        getTypeFactory().createTypeWithNullability(type, false);
-    return new RexCall(
-        typeNotNull,
-        SqlStdOperatorTable.CAST,
-        ImmutableList.of(expr));
-  }
-
-  /**
    * Creates a reference to all the fields in the row. That is, the whole row
    * as a single record object.
    *

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index e28653f..8af8a06 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -217,6 +217,9 @@ public class RexLiteral extends RexNode {
       return value instanceof Calendar;
     case INTERVAL_DAY_TIME:
     case INTERVAL_YEAR_MONTH:
+      // The value of a DAY-TIME interval (whatever the start and end units,
+      // even say HOUR TO MINUTE) is in milliseconds (perhaps fractional
+      // milliseconds). The value of a YEAR-MONTH interval is in months.
       return value instanceof BigDecimal;
     case VARBINARY: // not allowed -- use Binary
       if (strict) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
index fae09a5..e69dfc1 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexSqlStandardConvertletTable.java
@@ -176,24 +176,8 @@ public class RexSqlStandardConvertletTable
    *
    * @param op operator instance
    */
-  protected void registerEquivOp(final SqlOperator op) {
-    registerOp(
-        op,
-        new RexSqlConvertlet() {
-          public SqlNode convertCall(
-              RexToSqlNodeConverter converter,
-              RexCall call) {
-            SqlNode[] operands =
-                convertExpressionList(converter, call.operands);
-            if (operands == null) {
-              return null;
-            }
-            return new SqlBasicCall(
-                op,
-                operands,
-                SqlParserPos.ZERO);
-          }
-        });
+  protected void registerEquivOp(SqlOperator op) {
+    registerOp(op, new EquivConvertlet(op));
   }
 
   /**
@@ -265,6 +249,24 @@ public class RexSqlStandardConvertletTable
           }
         });
   }
+
+  /** Convertlet that converts a {@link SqlCall} to a {@link RexCall} of the
+   * same operator. */
+  private class EquivConvertlet implements RexSqlConvertlet {
+    private final SqlOperator op;
+
+    EquivConvertlet(SqlOperator op) {
+      this.op = op;
+    }
+
+    public SqlNode convertCall(RexToSqlNodeConverter converter, RexCall call) {
+      SqlNode[] operands = convertExpressionList(converter, call.operands);
+      if (operands == null) {
+        return null;
+      }
+      return new SqlBasicCall(op, operands, SqlParserPos.ZERO);
+    }
+  }
 }
 
 // End RexSqlStandardConvertletTable.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 e8da557..88c9356 100644
--- a/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
+++ b/core/src/main/java/org/apache/calcite/runtime/SqlFunctions.java
@@ -20,6 +20,7 @@ import org.apache.calcite.DataContext;
 import org.apache.calcite.avatica.util.ByteString;
 import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.Spaces;
+import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.linq4j.AbstractEnumerable;
 import org.apache.calcite.linq4j.CartesianProductEnumerator;
 import org.apache.calcite.linq4j.Enumerable;
@@ -579,6 +580,16 @@ public class SqlFunctions {
         : b0.divide(b1, MathContext.DECIMAL64);
   }
 
+  public static int divide(int b0, BigDecimal b1) {
+    return BigDecimal.valueOf(b0)
+        .divide(b1, BigDecimal.ROUND_HALF_DOWN).intValue();
+  }
+
+  public static long divide(long b0, BigDecimal b1) {
+    return BigDecimal.valueOf(b0)
+        .divide(b1, BigDecimal.ROUND_HALF_DOWN).longValue();
+  }
+
   // *
 
   /** SQL <code>*</code> operator applied to int values. */
@@ -1589,6 +1600,93 @@ public class SqlFunctions {
     };
   }
 
+  /** Adds a given number of months to a timestamp, represented as the number
+   * of milliseconds since the epoch. */
+  public static long addMonths(long timestamp, int m) {
+    final long millis =
+        DateTimeUtils.floorMod(timestamp, DateTimeUtils.MILLIS_PER_DAY);
+    timestamp -= millis;
+    final long x =
+        addMonths((int) (timestamp / DateTimeUtils.MILLIS_PER_DAY), m);
+    return x * DateTimeUtils.MILLIS_PER_DAY + millis;
+  }
+
+  /** Adds a given number of months to a date, represented as the number of
+   * days since the epoch. */
+  public static int addMonths(int date, int m) {
+    int y0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.YEAR, date);
+    int m0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.MONTH, date);
+    int d0 = (int) DateTimeUtils.unixDateExtract(TimeUnitRange.DAY, date);
+    int y = m / 12;
+    y0 += y;
+    m0 += m - y * 12;
+    int last = lastDay(y0, m0);
+    if (d0 > last) {
+      d0 = 1;
+      if (++m0 > 12) {
+        m0 = 1;
+        ++y0;
+      }
+    }
+    return DateTimeUtils.ymdToUnixDate(y0, m0, d0);
+  }
+
+  private static int lastDay(int y, int m) {
+    switch (m) {
+    case 2:
+      return y % 4 == 0
+          && (y % 100 != 0
+          || y % 400 == 0)
+          ? 29 : 28;
+    case 4:
+    case 6:
+    case 9:
+    case 11:
+      return 30;
+    default:
+      return 31;
+    }
+  }
+
+  /** 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) {
+    if (date0 < date1) {
+      return -subtractMonths(date1, date0);
+    }
+    // Start with an estimate.
+    // Since no month has more than 31 days, the estimate is <= the true value.
+    int m = (date0 - date1) / 31;
+    for (;;) {
+      int date2 = addMonths(date1, m);
+      if (date2 >= date0) {
+        return m;
+      }
+      int date3 = addMonths(date1, m + 1);
+      if (date3 > date0) {
+        return m;
+      }
+      ++m;
+    }
+  }
+
+  public static int subtractMonths(long t0, long t1) {
+    final long millis0 =
+        DateTimeUtils.floorMod(t0, DateTimeUtils.MILLIS_PER_DAY);
+    final int d0 = (int) DateTimeUtils.floorDiv(t0 - millis0,
+        DateTimeUtils.MILLIS_PER_DAY);
+    final long millis1 =
+        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);
+    final long d2 = addMonths(d1, x);
+    if (d2 == d0 && millis0 < millis1) {
+      --x;
+    }
+    return x;
+  }
+
   /** Enumerates over the cartesian product of the given lists, returning
    * a comparable list for each row. */
   private static class ProductComparableListEnumerator<E extends Comparable>

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 ae36bee..5255047 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -252,6 +252,12 @@ public class SqlIntervalQualifier extends SqlNode {
     return timeUnitRange.endUnit;
   }
 
+  /** Returns {@code SECOND} for both {@code HOUR TO SECOND} and
+   * {@code SECOND}. */
+  public TimeUnit getUnit() {
+    return Util.first(timeUnitRange.endUnit, timeUnitRange.startUnit);
+  }
+
   public SqlNode clone(SqlParserPos pos) {
     return new SqlIntervalQualifier(timeUnitRange.startUnit, startPrecision,
         timeUnitRange.endUnit, fractionalSecondPrecision, pos);

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
index f79c366..fd7e772 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlDatetimeSubtractionOperator.java
@@ -29,13 +29,15 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 
 /**
  * A special operator for the subtraction of two DATETIMEs. The format of
- * DATETIME substraction is:
+ * DATETIME subtraction is:
  *
  * <blockquote><code>"(" &lt;datetime&gt; "-" &lt;datetime&gt; ")"
  * &lt;interval qualifier&gt;</code></blockquote>
  *
  * <p>This operator is special since it needs to hold the
- * additional interval qualifier specification.</p>
+ * additional interval qualifier specification, when in {@link SqlCall} form.
+ * In {@link org.apache.calcite.rex.RexNode} form, it has only two parameters,
+ * and the return type describes the desired type of interval.
  */
 public class SqlDatetimeSubtractionOperator extends SqlSpecialOperator {
   //~ Constructors -----------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 1a8aea5..b6f4ed3 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
@@ -139,7 +139,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlKind.AND,
           28,
           true,
-          ReturnTypes.ARG0_NULLABLE, // more efficient than BOOLEAN_NULLABLE
+          ReturnTypes.BOOLEAN_NULLABLE_OPTIMIZED,
           InferTypes.BOOLEAN,
           OperandTypes.BOOLEAN_BOOLEAN);
 
@@ -428,7 +428,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlKind.OR,
           26,
           true,
-          ReturnTypes.ARG0_NULLABLE, // more efficient than BOOLEAN_NULLABLE
+          ReturnTypes.BOOLEAN_NULLABLE_OPTIMIZED,
           InferTypes.BOOLEAN,
           OperandTypes.BOOLEAN_BOOLEAN);
 
@@ -454,7 +454,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
           SqlKind.PLUS,
           40,
           true,
-          ReturnTypes.NULLABLE_SUM,
+          ReturnTypes.ARG0_NULLABLE,
           InferTypes.FIRST_KNOWN,
           OperandTypes.PLUS_OPERATOR);
 
@@ -896,7 +896,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
 
   /**
    * A special operator for the subtraction of two DATETIMEs. The format of
-   * DATETIME substraction is:
+   * DATETIME subtraction is:
    *
    * <blockquote><code>"(" &lt;datetime&gt; "-" &lt;datetime&gt; ")"
    * &lt;interval qualifier&gt;</code></blockquote>
@@ -904,7 +904,7 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <p>This operator is special since it needs to hold the
    * additional interval qualifier specification.</p>
    */
-  public static final SqlOperator MINUS_DATE =
+  public static final SqlDatetimeSubtractionOperator MINUS_DATE =
       new SqlDatetimeSubtractionOperator();
 
   /**
@@ -1347,8 +1347,8 @@ public class SqlStdOperatorTable extends ReflectiveSqlOperatorTable {
    * <p>Returns modified timestamp.
    */
   public static final SqlFunction TIMESTAMP_ADD =
-      new SqlFunction("TIMESTAMPADD", SqlKind.TIMESTAMP_ADD, ReturnTypes.ARG2,
-          null,
+      new SqlFunction("TIMESTAMPADD", SqlKind.TIMESTAMP_ADD,
+          ReturnTypes.ARG2_NULLABLE, null,
           OperandTypes.family(SqlTypeFamily.ANY, SqlTypeFamily.INTEGER,
               SqlTypeFamily.DATETIME), SqlFunctionCategory.TIMEDATE);
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
index f2083d0..6111c7b 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/IntervalSqlType.java
@@ -26,6 +26,8 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.pretty.SqlPrettyWriter;
 import org.apache.calcite.sql.util.SqlString;
 
+import com.google.common.base.Preconditions;
+
 /**
  * IntervalSqlType represents a standard SQL datetime interval type.
  */
@@ -33,7 +35,7 @@ public class IntervalSqlType extends AbstractSqlType {
   //~ Instance fields --------------------------------------------------------
 
   private final RelDataTypeSystem typeSystem;
-  private SqlIntervalQualifier intervalQualifier;
+  private final SqlIntervalQualifier intervalQualifier;
 
   //~ Constructors -----------------------------------------------------------
 
@@ -49,19 +51,17 @@ public class IntervalSqlType extends AbstractSqlType {
             : SqlTypeName.INTERVAL_DAY_TIME,
         isNullable,
         null);
-    this.typeSystem = typeSystem;
-    this.intervalQualifier = intervalQualifier;
+    this.typeSystem = Preconditions.checkNotNull(typeSystem);
+    this.intervalQualifier = Preconditions.checkNotNull(intervalQualifier);
     computeDigest();
   }
 
   //~ Methods ----------------------------------------------------------------
 
-  // implement RelDataTypeImpl
   protected void generateTypeString(StringBuilder sb, boolean withDetail) {
     sb.append("INTERVAL ");
-    SqlDialect dialect = null;
-    dialect = SqlDialect.DUMMY;
-    SqlPrettyWriter writer = new SqlPrettyWriter(dialect);
+    final SqlDialect dialect = SqlDialect.DUMMY;
+    final SqlPrettyWriter writer = new SqlPrettyWriter(dialect);
     writer.setAlwaysUseParentheses(false);
     writer.setSelectListItemsOnSeparateLines(false);
     writer.setIndentation(0);
@@ -70,8 +70,7 @@ public class IntervalSqlType extends AbstractSqlType {
     sb.append(new SqlString(dialect, sql).getSql());
   }
 
-  // implement RelDataType
-  public SqlIntervalQualifier getIntervalQualifier() {
+  @Override public SqlIntervalQualifier getIntervalQualifier() {
     return intervalQualifier;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
----------------------------------------------------------------------
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 a81caf3..62c9ad3 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
@@ -104,23 +104,7 @@ public abstract class ReturnTypes {
    * returned type will also be nullable.
    */
   public static final SqlReturnTypeInference ARG0_NULLABLE =
-      new SqlReturnTypeInference() {
-        // Equivalent to
-        //   cascade(ARG0, SqlTypeTransforms.TO_NULLABLE);
-        // but implemented by hand because used in AND, which is a very common
-        // operator.
-        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
-          final int n = opBinding.getOperandCount();
-          RelDataType type1 = null;
-          for (int i = 0; i < n; i++) {
-            type1 = opBinding.getOperandType(i);
-            if (type1.isNullable()) {
-              break;
-            }
-          }
-          return type1;
-        }
-      };
+      cascade(ARG0, SqlTypeTransforms.TO_NULLABLE);
 
   /**
    * Type-inference strategy whereby the result type of a call is the type of
@@ -193,6 +177,31 @@ public abstract class ReturnTypes {
    */
   public static final SqlReturnTypeInference BOOLEAN_NULLABLE =
       cascade(BOOLEAN, SqlTypeTransforms.TO_NULLABLE);
+
+  /**
+   * Type-inference strategy with similar effect to {@link #BOOLEAN_NULLABLE},
+   * which is more efficient, but can only be used if all arguments are
+   * BOOLEAN.
+   */
+  public static final SqlReturnTypeInference BOOLEAN_NULLABLE_OPTIMIZED =
+      new SqlReturnTypeInference() {
+        // Equivalent to
+        //   cascade(ARG0, SqlTypeTransforms.TO_NULLABLE);
+        // but implemented by hand because used in AND, which is a very common
+        // operator.
+        public RelDataType inferReturnType(SqlOperatorBinding opBinding) {
+          final int n = opBinding.getOperandCount();
+          RelDataType type1 = null;
+          for (int i = 0; i < n; i++) {
+            type1 = opBinding.getOperandType(i);
+            if (type1.isNullable()) {
+              break;
+            }
+          }
+          return type1;
+        }
+      };
+
   /**
    * Type-inference strategy whereby the result type of a call is Boolean
    * not null.

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 643d1c5..c9db0f3 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/StandardConvertletTable.java
@@ -75,7 +75,6 @@ import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
 
 import java.math.BigDecimal;
-import java.math.MathContext;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Set;
@@ -137,6 +136,24 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
           }
         });
 
+    registerOp(SqlStdOperatorTable.MINUS,
+        new SqlRexConvertlet() {
+          public RexNode convertCall(SqlRexContext cx, SqlCall call) {
+            final RexCall e =
+                (RexCall) StandardConvertletTable.this.convertCall(cx, call,
+                    call.getOperator());
+            switch (e.getOperands().get(0).getType().getSqlTypeName()) {
+            case DATE:
+            case TIME:
+            case TIMESTAMP:
+              return convertDatetimeMinus(cx, SqlStdOperatorTable.MINUS_DATE,
+                  call);
+            default:
+              return e;
+            }
+          }
+        });
+
     registerOp(OracleSqlOperatorTable.LTRIM,
         new TrimConvertlet(SqlTrimFunction.Flag.LEADING));
     registerOp(OracleSqlOperatorTable.RTRIM,
@@ -172,7 +189,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
                 call.getOperandList(), SqlOperandTypeChecker.Consistency.NONE);
             final RelDataType type =
                 cx.getValidator().getValidatedNodeType(call);
-            final List<RexNode> exprs = new ArrayList<RexNode>();
+            final List<RexNode> exprs = new ArrayList<>();
             for (int i = 1; i < operands.size() - 1; i += 2) {
               exprs.add(
                   RelOptUtil.isDistinctFrom(rexBuilder, operands.get(0),
@@ -458,17 +475,25 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     if (right instanceof SqlIntervalQualifier) {
       final SqlIntervalQualifier intervalQualifier =
           (SqlIntervalQualifier) right;
-      if (left instanceof SqlIntervalLiteral
-          || left instanceof SqlNumericLiteral) {
+      if (left instanceof SqlIntervalLiteral) {
         RexLiteral sourceInterval =
             (RexLiteral) cx.convertExpression(left);
         BigDecimal sourceValue =
             (BigDecimal) sourceInterval.getValue();
         RexLiteral castedInterval =
+            cx.getRexBuilder().makeIntervalLiteral(sourceValue,
+                intervalQualifier);
+        return castToValidatedType(cx, call, castedInterval);
+      } else if (left instanceof SqlNumericLiteral) {
+        RexLiteral sourceInterval =
+            (RexLiteral) cx.convertExpression(left);
+        BigDecimal sourceValue =
+            (BigDecimal) sourceInterval.getValue();
+        final BigDecimal multiplier = intervalQualifier.getUnit().multiplier;
+        sourceValue = sourceValue.multiply(multiplier);
+        RexLiteral castedInterval =
             cx.getRexBuilder().makeIntervalLiteral(
-                sourceValue.multiply(
-                    intervalQualifier.getStartUnit().multiplier,
-                    MathContext.UNLIMITED),
+                sourceValue,
                 intervalQualifier);
         return castToValidatedType(cx, call, castedInterval);
       }
@@ -746,7 +771,8 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
         rexBuilder.makeExactLiteral(val, resType));
   }
 
-  private RexNode divide(RexBuilder rexBuilder, RexNode res, BigDecimal val) {
+  private static RexNode divide(RexBuilder rexBuilder, RexNode res,
+      BigDecimal val) {
     if (val.equals(BigDecimal.ONE)) {
       return res;
     }
@@ -777,14 +803,6 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     final List<RexNode> exprs = convertExpressionList(cx, operands,
         SqlOperandTypeChecker.Consistency.NONE);
 
-    // TODO: Handle year month interval (represented in months)
-    for (RexNode expr : exprs) {
-      if (SqlTypeName.INTERVAL_YEAR_MONTH
-          == expr.getType().getSqlTypeName()) {
-        throw Util.needToImplement(
-            "Datetime subtraction of year month interval");
-      }
-    }
     RelDataType int8Type =
         cx.getTypeFactory().createSqlType(SqlTypeName.BIGINT);
     final RexNode[] casts = new RexNode[2];
@@ -800,16 +818,9 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
                 int8Type,
                 exprs.get(1).getType().isNullable()),
             exprs.get(1));
-    final RexNode minus =
-        rexBuilder.makeCall(
-            SqlStdOperatorTable.MINUS,
-            casts);
     final RelDataType resType =
         cx.getValidator().getValidatedNodeType(call);
-    return rexBuilder.makeReinterpretCast(
-        resType,
-        minus,
-        rexBuilder.makeLiteral(false));
+    return rexBuilder.makeCall(resType, op, exprs.subList(0, 2));
   }
 
   public RexNode convertFunction(
@@ -1014,7 +1025,20 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
     case DATE:
     case TIME:
     case TIMESTAMP:
-      return convertCall(cx, call, SqlStdOperatorTable.DATETIME_PLUS);
+      // Use special "+" operator for datetime + interval.
+      // Re-order operands, if necessary, so that interval is second.
+      final RexBuilder rexBuilder = cx.getRexBuilder();
+      List<RexNode> operands = ((RexCall) rex).getOperands();
+      if (operands.size() == 2) {
+        final SqlTypeName sqlTypeName = operands.get(0).getType().getSqlTypeName();
+        switch (sqlTypeName) {
+        case INTERVAL_DAY_TIME:
+        case INTERVAL_YEAR_MONTH:
+          operands = ImmutableList.of(operands.get(1), operands.get(0));
+        }
+      }
+      return rexBuilder.makeCall(rex.getType(),
+          SqlStdOperatorTable.DATETIME_PLUS, operands);
     default:
       return rex;
     }
@@ -1421,7 +1445,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
   }
 
   /** Convertlet that handles the {@code TIMESTAMPADD} function. */
-  private class TimestampAddConvertlet implements SqlRexConvertlet {
+  private static class TimestampAddConvertlet implements SqlRexConvertlet {
     public RexNode convertCall(SqlRexContext cx, SqlCall call) {
       // TIMESTAMPADD(unit, count, timestamp)
       //  => timestamp + count * INTERVAL '1' UNIT
@@ -1439,7 +1463,7 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
   }
 
   /** Convertlet that handles the {@code TIMESTAMPDIFF} function. */
-  private class TimestampDiffConvertlet implements SqlRexConvertlet {
+  private static class TimestampDiffConvertlet implements SqlRexConvertlet {
     public RexNode convertCall(SqlRexContext cx, SqlCall call) {
       // TIMESTAMPDIFF(unit, t1, t2)
       //    => (t2 - t1) UNIT
@@ -1450,13 +1474,12 @@ public class StandardConvertletTable extends ReflectiveConvertletTable {
           cx.getTypeFactory().createSqlType(SqlTypeName.INTEGER);
       final SqlIntervalQualifier qualifier =
           new SqlIntervalQualifier(unit, null, SqlParserPos.ZERO);
-      return divide(cx.getRexBuilder(),
-          rexBuilder.makeCast(intType,
-              rexBuilder.makeCall(SqlStdOperatorTable.MINUS_DATE,
-                  cx.convertExpression(call.operand(2)),
-                  cx.convertExpression(call.operand(1)),
-                  cx.getRexBuilder().makeIntervalLiteral(qualifier))),
-          unit.multiplier);
+      final RelDataType intervalType =
+          cx.getTypeFactory().createSqlIntervalType(qualifier);
+      return rexBuilder.makeCast(intType,
+          rexBuilder.makeCall(intervalType, SqlStdOperatorTable.MINUS_DATE,
+              ImmutableList.of(cx.convertExpression(call.operand(2)),
+                  cx.convertExpression(call.operand(1)))));
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
index 9850c9f..9d44bd7 100644
--- a/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
+++ b/core/src/main/java/org/apache/calcite/util/BuiltInMethod.java
@@ -250,6 +250,9 @@ public enum BuiltInMethod {
   STRING_CONCAT(SqlFunctions.class, "concat", String.class, String.class),
   FLOOR_DIV(DateTimeUtils.class, "floorDiv", long.class, long.class),
   FLOOR_MOD(DateTimeUtils.class, "floorMod", long.class, long.class),
+  ADD_MONTHS(SqlFunctions.class, "addMonths", long.class, int.class),
+  SUBTRACT_MONTHS(SqlFunctions.class, "subtractMonths", long.class,
+      long.class),
   FLOOR(SqlFunctions.class, "floor", int.class, int.class),
   CEIL(SqlFunctions.class, "ceil", int.class, int.class),
   OVERLAY(SqlFunctions.class, "overlay", String.class, String.class, int.class),

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 030e2f3..e81ed4c 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
@@ -272,11 +272,6 @@ public abstract class SqlOperatorBaseTest {
    */
   public static final boolean DECIMAL = false;
 
-  /**
-   * Whether INTERVAL type is implemented.
-   */
-  public static final boolean INTERVAL = false;
-
   private final boolean enable;
 
   protected final SqlTester tester;
@@ -344,7 +339,6 @@ public abstract class SqlOperatorBaseTest {
     }
   }
 
-
   @Test public void testBetween() {
     tester.setFor(
         SqlStdOperatorTable.BETWEEN,
@@ -820,10 +814,6 @@ public abstract class SqlOperatorBaseTest {
           "-5.0");
     }
 
-    if (!INTERVAL) {
-      return;
-    }
-
     // Interval to bigint
     tester.checkScalarExact(
         "cast(INTERVAL '1.25' second as bigint)",
@@ -855,9 +845,6 @@ public abstract class SqlOperatorBaseTest {
 
   @Test public void testCastToInterval() {
     tester.setFor(SqlStdOperatorTable.CAST);
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "cast(5 as interval second)",
         "+5.000000",
@@ -882,14 +869,23 @@ public abstract class SqlOperatorBaseTest {
         "cast(5 as interval year)",
         "+5",
         "INTERVAL YEAR NOT NULL");
-    tester.checkScalar(
-        "cast(5.7 as interval day)",
-        "+6",
-        "INTERVAL DAY NOT NULL");
-    tester.checkScalar(
-        "cast(-5.7 as interval day)",
-        "-6",
-        "INTERVAL DAY NOT NULL");
+    if (DECIMAL) {
+      // Due to DECIMAL rounding bugs, currently returns "+5"
+      tester.checkScalar(
+          "cast(5.7 as interval day)",
+          "+6",
+          "INTERVAL DAY NOT NULL");
+      tester.checkScalar(
+          "cast(-5.7 as interval day)",
+          "-6",
+          "INTERVAL DAY NOT NULL");
+    } else {
+      // An easier case
+      tester.checkScalar(
+          "cast(6.2 as interval day)",
+          "+6",
+          "INTERVAL DAY NOT NULL");
+    }
     tester.checkScalar(
         "cast(3456 as interval month(4))",
         "+3456",
@@ -901,13 +897,26 @@ public abstract class SqlOperatorBaseTest {
   }
 
   @Test public void testCastIntervalToInterval() {
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "cast(interval '2 5' day to hour as interval hour to minute)",
-        "+29:00",
+        "+53:00",
         "INTERVAL HOUR TO MINUTE NOT NULL");
+    tester.checkScalar(
+        "cast(interval '2 5' day to hour as interval day to minute)",
+        "+2 05:00",
+        "INTERVAL DAY TO MINUTE NOT NULL");
+    tester.checkScalar(
+        "cast(interval '2 5' day to hour as interval hour to second)",
+        "+53:00:00.000000",
+        "INTERVAL HOUR TO SECOND NOT NULL");
+    tester.checkScalar(
+        "cast(interval '2 5' day to hour as interval hour)",
+        "+53",
+        "INTERVAL HOUR NOT NULL");
+    tester.checkScalar(
+        "cast(interval '-29:15' hour to minute as interval day to hour)",
+        "-1 05",
+        "INTERVAL DAY TO HOUR NOT NULL");
   }
 
   @Test public void testCastWithRoundingToScalar() {
@@ -1967,16 +1976,13 @@ public abstract class SqlOperatorBaseTest {
         "interval '2' day / cast(null as bigint)");
     tester.checkNull(
         "cast(null as interval month) / 2");
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "interval '3-3' year to month / 15e-1",
-        "+02-02",
+        "+2-02",
         "INTERVAL YEAR TO MONTH NOT NULL");
     tester.checkScalar(
         "interval '3-4' year to month / 4.5",
-        "+00-08",
+        "+0-09",
         "INTERVAL YEAR TO MONTH NOT NULL");
   }
 
@@ -2550,9 +2556,6 @@ public abstract class SqlOperatorBaseTest {
         "time '12:03:01' - interval '1:1' hour to minute",
         "11:02:01",
         "TIME(0) NOT NULL");
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "date '2005-03-02' - interval '5' day",
         "2005-02-25",
@@ -2562,17 +2565,26 @@ public abstract class SqlOperatorBaseTest {
         "2003-08-06 14:58:01",
         "TIMESTAMP(0) NOT NULL");
 
-    // TODO: Tests with interval year months (not supported)
+    // Datetime minus year-month interval
+    tester.checkScalar(
+        "timestamp '2003-08-02 12:54:01' - interval '12' year",
+        "1991-08-02 12:54:01",
+        "TIMESTAMP(0) NOT NULL");
+    tester.checkScalar(
+        "date '2003-08-02' - interval '12' year",
+        "1991-08-02",
+        "DATE NOT NULL");
+    tester.checkScalar(
+        "date '2003-08-02' - interval '12-3' year to month",
+        "1991-05-02",
+        "DATE NOT NULL");
   }
 
   @Test public void testMinusDateOperator() {
     tester.setFor(SqlStdOperatorTable.MINUS_DATE);
-    if (!enable) {
-      return;
-    }
     tester.checkScalar(
         "(time '12:03:34' - time '11:57:23') minute to second",
-        "+6:11",
+        "+6:11.000000",
         "INTERVAL MINUTE TO SECOND NOT NULL");
     tester.checkScalar(
         "(time '12:03:23' - time '11:57:23') minute",
@@ -2584,7 +2596,7 @@ public abstract class SqlOperatorBaseTest {
         "INTERVAL MINUTE NOT NULL");
     tester.checkScalar(
         "(timestamp '2004-05-01 12:03:34' - timestamp '2004-04-29 11:57:23') day to second",
-        "+2 00:06:11",
+        "+2 00:06:11.000000",
         "INTERVAL DAY TO SECOND NOT NULL");
     tester.checkScalar(
         "(timestamp '2004-05-01 12:03:34' - timestamp '2004-04-29 11:57:23') day to hour",
@@ -2889,9 +2901,6 @@ public abstract class SqlOperatorBaseTest {
         "time '12:03:01' + interval '1:1' hour to minute",
         "13:04:01",
         "TIME(0) NOT NULL");
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "interval '5' day + date '2005-03-02'",
         "2005-03-07",
@@ -2901,7 +2910,15 @@ public abstract class SqlOperatorBaseTest {
         "2003-07-29 10:50:01",
         "TIMESTAMP(0) NOT NULL");
 
-    // TODO: Tests with interval year months (not supported)
+    // Datetime plus year-to-month interval
+    tester.checkScalar(
+        "interval '5-3' year to month + date '2005-03-02'",
+        "2010-06-02",
+        "DATE NOT NULL");
+    tester.checkScalar(
+        "timestamp '2003-08-02 12:54:01' + interval '5-3' year to month",
+        "2008-11-02 12:54:01",
+        "TIMESTAMP(0) NOT NULL");
   }
 
   @Test public void testDescendingOperator() {
@@ -3848,9 +3865,6 @@ public abstract class SqlOperatorBaseTest {
         "nullif(interval '2' month, interval '3' year)",
         "+2",
         "INTERVAL MONTH");
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "nullif(interval '2 5' day to hour, interval '5' second)",
         "+2 05",
@@ -4734,9 +4748,6 @@ public abstract class SqlOperatorBaseTest {
         "extract(year from interval '4-2' year to month)",
         "4",
         "BIGINT NOT NULL");
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkScalar(
         "extract(month from interval '4-2' year to month)",
         "2",
@@ -5038,9 +5049,6 @@ public abstract class SqlOperatorBaseTest {
         "timestampadd(HOUR, -2000, timestamp '2016-02-24 12:42:25')",
         "2015-12-03 04:42:25",
         "TIMESTAMP(0) NOT NULL");
-    if (!INTERVAL) {
-      return;
-    }
     tester.checkNull("timestampadd(HOUR, CAST(NULL AS INTEGER),"
         + " timestamp '2016-02-24 12:42:25')");
     tester.checkNull(
@@ -5060,9 +5068,6 @@ public abstract class SqlOperatorBaseTest {
         + "timestamp '2016-02-24 12:42:25', "
         + "timestamp '2016-02-24 12:42:20')",
         "-5000000", "INTEGER NOT NULL");
-    if (!INTERVAL) {
-      return;
-    }
     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/5511253b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/JdbcTest.java b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
index fc2c396..172e66a 100644
--- a/core/src/test/java/org/apache/calcite/test/JdbcTest.java
+++ b/core/src/test/java/org/apache/calcite/test/JdbcTest.java
@@ -3764,8 +3764,6 @@ public class JdbcTest {
   }
 
   /** Tests for DATE +- INTERVAL window frame */
-  @Ignore("DATE/TIMESTAMP/INTERVAL support is broken:"
-      + "1 year is converted to 12 months instead of milliseconds")
   @Test public void testWinIntervalFrame() {
     CalciteAssert.hr()
         .query("select  \"deptno\",\n"
@@ -3777,12 +3775,11 @@ public class JdbcTest {
             + "  DATE '2014-06-12' + \"empid\"*interval '0' day \"hire_date\"\n"
             + "  from \"hr\".\"emps\")")
         .typeIs(
-            "[deptno INTEGER NOT NULL, empid INTEGER NOT NULL, hire_date DATE NOT NULL, R BIGINT]")
-        .returnsUnordered(
-            "deptno=10; R=1",
-            "deptno=10; R=1",
-            "deptno=10; R=1",
-            "deptno=20; R=4"); // 4 for rank and 2 for dense_rank
+            "[deptno INTEGER NOT NULL, empid INTEGER NOT NULL, hire_date DATE NOT NULL, R BIGINT NOT NULL]")
+        .returnsUnordered("deptno=10; empid=100; hire_date=2014-06-12; R=3",
+            "deptno=10; empid=110; hire_date=2014-06-12; R=3",
+            "deptno=10; empid=150; hire_date=2014-06-12; R=3",
+            "deptno=20; empid=200; hire_date=2014-06-12; R=1");
   }
 
   private void startOfGroupStep1(String startOfGroup) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/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 88ab545..241ecb7 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlFunctionsTest.java
@@ -17,6 +17,7 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.runtime.SqlFunctions;
 import org.apache.calcite.runtime.Utilities;
@@ -45,6 +46,7 @@ import static org.apache.calcite.avatica.util.DateTimeUtils.unixTimestamp;
 import static org.apache.calcite.avatica.util.DateTimeUtils.unixTimestampToString;
 import static org.apache.calcite.avatica.util.DateTimeUtils.ymdToJulian;
 import static org.apache.calcite.avatica.util.DateTimeUtils.ymdToUnixDate;
+import static org.apache.calcite.runtime.SqlFunctions.addMonths;
 import static org.apache.calcite.runtime.SqlFunctions.charLength;
 import static org.apache.calcite.runtime.SqlFunctions.concat;
 import static org.apache.calcite.runtime.SqlFunctions.greater;
@@ -53,9 +55,11 @@ import static org.apache.calcite.runtime.SqlFunctions.lesser;
 import static org.apache.calcite.runtime.SqlFunctions.lower;
 import static org.apache.calcite.runtime.SqlFunctions.ltrim;
 import static org.apache.calcite.runtime.SqlFunctions.rtrim;
+import static org.apache.calcite.runtime.SqlFunctions.subtractMonths;
 import static org.apache.calcite.runtime.SqlFunctions.trim;
 import static org.apache.calcite.runtime.SqlFunctions.upper;
 
+import static org.hamcrest.CoreMatchers.anyOf;
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
 import static org.junit.Assert.assertEquals;
@@ -355,6 +359,45 @@ public class SqlFunctionsTest {
         equalTo((long) day));
   }
 
+  @Test public void testAddMonths() {
+    checkAddMonths(2016, 1, 1, 2016, 2, 1, 1);
+    checkAddMonths(2016, 1, 1, 2017, 1, 1, 12);
+    checkAddMonths(2016, 1, 1, 2017, 2, 1, 13);
+    checkAddMonths(2016, 1, 1, 2015, 1, 1, -12);
+    checkAddMonths(2016, 1, 1, 2018, 10, 1, 33);
+    checkAddMonths(2016, 1, 31, 2016, 5, 1, 3); // roll up
+    checkAddMonths(2016, 4, 30, 2016, 7, 30, 3); // roll up
+    checkAddMonths(2016, 1, 31, 2016, 3, 1, 1);
+    checkAddMonths(2016, 3, 31, 2016, 3, 1, -1);
+    checkAddMonths(2016, 3, 31, 2116, 3, 31, 1200);
+    checkAddMonths(2016, 2, 28, 2116, 2, 28, 1200);
+  }
+
+  private void checkAddMonths(int y0, int m0, int d0, int y1, int m1, int d1,
+      int months) {
+    final int date0 = ymdToUnixDate(y0, m0, d0);
+    final long date = addMonths(date0, months);
+    final int date1 = ymdToUnixDate(y1, m1, d1);
+    assertThat((int) date, is(date1));
+
+    assertThat(subtractMonths(date1, date0),
+        anyOf(is((long) months), is((long) months + 1)));
+    assertThat(subtractMonths(date1 + 1, date0),
+        anyOf(is((long) months), is((long) months + 1)));
+    assertThat(subtractMonths(date1, date0 + 1),
+        anyOf(is((long) months), is((long) months - 1)));
+    assertThat(subtractMonths(d2ts(date1, 1), d2ts(date0, 0)),
+        anyOf(is(months), is(months + 1)));
+    assertThat(subtractMonths(d2ts(date1, 0), d2ts(date0, 1)),
+        anyOf(is(months - 1), is(months), is(months + 1)));
+  }
+
+  /** Converts a date (days since epoch) and milliseconds (since midnight)
+   * into a timestamp (milliseconds since epoch). */
+  private long d2ts(int date, int millis) {
+    return date * DateTimeUtils.MILLIS_PER_DAY + millis;
+  }
+
   @Test public void testUnixTimestamp() {
     assertThat(unixTimestamp(1970, 1, 1, 0, 0, 0), is(0L));
     final long day = 86400000L;

http://git-wip-us.apache.org/repos/asf/calcite/blob/5511253b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
index 3b2f92d..f95c971 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -3714,7 +3714,7 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
         "cast(1 as DECIMAL(19, 2)) + cast(1 as DECIMAL(19, 2))",
         "DECIMAL(19, 2) NOT NULL");
 
-    // substraction operator
+    // subtraction operator
     checkExpType(
         "cast(1 as TINYINT) - cast(5 as BIGINT)",
         "BIGINT NOT NULL");