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/10/18 19:04:46 UTC

[1/3] calcite git commit: [CALCITE-1417] In RelBuilder, simplify "CAST(literal TO type)" to a literal when possible

Repository: calcite
Updated Branches:
  refs/heads/master d9e9103bb -> 105bba1f8


http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
index 70917e8..7f6eee4 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidRules.java
@@ -16,6 +16,7 @@
  */
 package org.apache.calcite.adapter.druid;
 
+import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptRule;
 import org.apache.calcite.plan.RelOptRuleCall;
 import org.apache.calcite.plan.RelOptUtil;
@@ -100,6 +101,8 @@ public class DruidRules {
     public void onMatch(RelOptRuleCall call) {
       final Filter filter = call.rel(0);
       final DruidQuery query = call.rel(1);
+      final RelOptCluster cluster = filter.getCluster();
+      final RexBuilder rexBuilder = cluster.getRexBuilder();
       if (!DruidQuery.isValidSignature(query.signature() + 'f')
               || !query.isValidFilter(filter.getCondition())) {
         return;
@@ -113,8 +116,9 @@ public class DruidRules {
           break;
         }
       }
-      final Pair<List<RexNode>, List<RexNode>> pair = splitFilters(
-              filter.getCluster().getRexBuilder(), query, filter.getCondition(),
+      final Pair<List<RexNode>, List<RexNode>> pair =
+          splitFilters(rexBuilder, query,
+              RexUtil.simplify(rexBuilder, filter.getCondition(), true),
               timestampFieldIdx);
       if (pair == null) {
         // We can't push anything useful to Druid.
@@ -124,7 +128,7 @@ public class DruidRules {
       if (!pair.left.isEmpty()) {
         intervals = DruidDateTimeUtils.createInterval(
                 query.getRowType().getFieldList().get(timestampFieldIdx).getType(),
-                RexUtil.composeConjunction(query.getCluster().getRexBuilder(), pair.left, false));
+                RexUtil.composeConjunction(rexBuilder, pair.left, false));
         if (intervals == null) {
           // We can't push anything useful to Druid.
           return;
@@ -133,7 +137,7 @@ public class DruidRules {
       DruidQuery newDruidQuery = query;
       if (!pair.right.isEmpty()) {
         final RelNode newFilter = filter.copy(filter.getTraitSet(), Util.last(query.rels),
-                RexUtil.composeConjunction(filter.getCluster().getRexBuilder(), pair.right, false));
+            RexUtil.composeConjunction(rexBuilder, pair.right, false));
         newDruidQuery = DruidQuery.extendQuery(query, newFilter);
       }
       if (intervals != null) {
@@ -189,6 +193,8 @@ public class DruidRules {
     public void onMatch(RelOptRuleCall call) {
       final Project project = call.rel(0);
       final DruidQuery query = call.rel(1);
+      final RelOptCluster cluster = project.getCluster();
+      final RexBuilder rexBuilder = cluster.getRexBuilder();
       if (!DruidQuery.isValidSignature(query.signature() + 'p')) {
         return;
       }
@@ -201,16 +207,16 @@ public class DruidRules {
         call.transformTo(newNode);
         return;
       }
-      final Pair<List<RexNode>, List<RexNode>> pair = splitProjects(
-              project.getCluster().getRexBuilder(), query, project.getProjects());
+      final Pair<List<RexNode>, List<RexNode>> pair =
+          splitProjects(rexBuilder, query, project.getProjects());
       if (pair == null) {
         // We can't push anything useful to Druid.
         return;
       }
       final List<RexNode> above = pair.left;
       final List<RexNode> below = pair.right;
-      final RelDataTypeFactory.FieldInfoBuilder builder = project.getCluster().getTypeFactory()
-              .builder();
+      final RelDataTypeFactory.FieldInfoBuilder builder =
+          cluster.getTypeFactory().builder();
       final RelNode input = Util.last(query.rels);
       for (RexNode e : below) {
         final String name;


[2/3] calcite git commit: [CALCITE-1417] In RelBuilder, simplify "CAST(literal TO type)" to a literal when possible

Posted by jh...@apache.org.
[CALCITE-1417] In RelBuilder, simplify "CAST(literal TO type)" to a literal when possible

This allows us to simplify logic that the Druid adapter uses to push
down time ranges. For example, CAST('2016-01-23' AS DATE) will already
be a DATE value.

You can turn off simplification (mainly for testing purposes) by
setting Hook.REL_BUILDER_SIMPLIFY. This the first time we have a
allowed a Hook to act like a property.

Add a test case (disabled) for [CALCITE-1439].

Allow QuidemTest to accept test script names as command-line
arguments.

Use RexExecutor for constant reduction (although not necessarily the
same one that will be used later in planning).


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

Branch: refs/heads/master
Commit: 54556b82c0599012f2e7fefaae3a9868cb68f879
Parents: d9e9103
Author: Julian Hyde <jh...@apache.org>
Authored: Sun Oct 9 16:21:47 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Oct 18 10:20:28 2016 -0700

----------------------------------------------------------------------
 .../java/org/apache/calcite/rex/RexBuilder.java |  11 +-
 .../java/org/apache/calcite/rex/RexUtil.java    |  81 ++++-
 .../java/org/apache/calcite/runtime/Hook.java   |  14 +
 .../apache/calcite/sql/fun/SqlCastFunction.java |  90 ++----
 .../org/apache/calcite/tools/RelBuilder.java    |  14 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |  36 ++-
 .../org/apache/calcite/test/QuidemTest.java     |  11 +-
 .../apache/calcite/test/RelMetadataTest.java    |   3 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  56 +++-
 .../org/apache/calcite/test/RelOptTestBase.java |  65 +++-
 .../org/apache/calcite/test/RexProgramTest.java | 116 ++++++++
 .../org/apache/calcite/test/RelOptRulesTest.xml |  13 +-
 .../calcite/test/SqlToRelConverterTest.xml      | 178 +++++++----
 core/src/test/resources/sql/dummy.iq            |  15 +-
 core/src/test/resources/sql/misc.iq             | 220 ++++++++++++++
 .../adapter/druid/DruidDateTimeUtils.java       | 295 +++++--------------
 .../calcite/adapter/druid/DruidRules.java       |  22 +-
 17 files changed, 838 insertions(+), 402 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/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 73ef60d..87c84e3 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -52,6 +52,7 @@ import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import java.math.BigDecimal;
+import java.math.MathContext;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Calendar;
@@ -577,7 +578,7 @@ public class RexBuilder {
     }
   }
 
-  private boolean canRemoveCastFromLiteral(RelDataType toType, Comparable value,
+  boolean canRemoveCastFromLiteral(RelDataType toType, Comparable value,
       SqlTypeName fromTypeName) {
     final SqlTypeName sqlType = toType.getSqlTypeName();
     if (!RexLiteral.valueMatchesType(value, sqlType, false)) {
@@ -1375,12 +1376,18 @@ public class RexBuilder {
       }
       return new BigDecimal(((Number) o).longValue());
     case FLOAT:
+      if (o instanceof BigDecimal) {
+        return o;
+      }
+      return new BigDecimal(((Number) o).doubleValue(), MathContext.DECIMAL32)
+          .stripTrailingZeros();
     case REAL:
     case DOUBLE:
       if (o instanceof BigDecimal) {
         return o;
       }
-      return new BigDecimal(((Number) o).doubleValue());
+      return new BigDecimal(((Number) o).doubleValue(), MathContext.DECIMAL64)
+          .stripTrailingZeros();
     case CHAR:
     case VARCHAR:
       if (o instanceof NlsString) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/main/java/org/apache/calcite/rex/RexUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexUtil.java b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
index d2a95fa..fd971bd 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -18,6 +18,7 @@ package org.apache.calcite.rex;
 
 import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.linq4j.function.Predicate1;
+import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.RelCollation;
 import org.apache.calcite.rel.RelCollations;
@@ -29,6 +30,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rel.type.RelDataTypeFamily;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.schema.Schemas;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
@@ -43,6 +45,7 @@ import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
 
 import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.ArrayListMultimap;
 import com.google.common.collect.ImmutableList;
@@ -104,6 +107,11 @@ public class RexUtil {
         }
       };
 
+  /** Executor for a bit of constant reduction. Ideally we'd use the user's
+   * preferred executor, but that isn't available. */
+  private static final RelOptPlanner.Executor EXECUTOR =
+      new RexExecutorImpl(Schemas.createDataContext(null));
+
   private RexUtil() {
   }
 
@@ -1386,6 +1394,24 @@ public class RexUtil {
     return e1 == e2 || e1.toString().equals(e2.toString());
   }
 
+  /** Simplifies a boolean expression, always preserving its type and its
+   * nullability.
+   *
+   * <p>This is useful if you are simplifying expressions in a
+   * {@link Project}. */
+  public static RexNode simplifyPreservingType(RexBuilder rexBuilder,
+      RexNode e) {
+    final RexNode e2 = simplify(rexBuilder, e, false);
+    if (e2.getType() == e.getType()) {
+      return e2;
+    }
+    final RexNode e3 = rexBuilder.makeCast(e.getType(), e2, true);
+    if (e3.equals(e)) {
+      return e;
+    }
+    return e3;
+  }
+
   /**
    * Simplifies a boolean expression.
    *
@@ -1412,23 +1438,37 @@ public class RexUtil {
       return simplifyNot(rexBuilder, (RexCall) e);
     case CASE:
       return simplifyCase(rexBuilder, (RexCall) e, unknownAsFalse);
+    case CAST:
+      return simplifyCast(rexBuilder, (RexCall) e);
     case IS_NULL:
-      return ((RexCall) e).getOperands().get(0).getType().isNullable()
-          ? e : rexBuilder.makeLiteral(false);
     case IS_NOT_NULL:
-      return ((RexCall) e).getOperands().get(0).getType().isNullable()
-          ? e : rexBuilder.makeLiteral(true);
     case IS_TRUE:
     case IS_NOT_TRUE:
     case IS_FALSE:
     case IS_NOT_FALSE:
       assert e instanceof RexCall;
       return simplifyIs(rexBuilder, (RexCall) e);
+    case EQUALS:
+    case GREATER_THAN:
+    case GREATER_THAN_OR_EQUAL:
+    case LESS_THAN:
+    case LESS_THAN_OR_EQUAL:
+    case NOT_EQUALS:
+      return simplifyCall(rexBuilder, (RexCall) e);
     default:
       return e;
     }
   }
 
+  private static RexNode simplifyCall(RexBuilder rexBuilder, RexCall e) {
+    final List<RexNode> operands = new ArrayList<>(e.operands);
+    simplifyList(rexBuilder, operands);
+    if (operands.equals(e.operands)) {
+      return e;
+    }
+    return rexBuilder.makeCall(e.op, operands);
+  }
+
   /**
    * Simplifies a conjunction of boolean expressions.
    */
@@ -2074,6 +2114,39 @@ public class RexUtil {
         && (call.operands.size() - i) % 2 == 1;
   }
 
+  private static RexNode simplifyCast(RexBuilder rexBuilder, RexCall e) {
+    final RexNode operand = e.getOperands().get(0);
+    switch (operand.getKind()) {
+    case LITERAL:
+      final RexLiteral literal = (RexLiteral) operand;
+      final Comparable value = literal.getValue();
+      final SqlTypeName typeName = literal.getTypeName();
+
+      // First, try to remove the cast without changing the value.
+      // makeCast and canRemoveCastFromLiteral have the same logic, so we are
+      // sure to be able to remove the cast.
+      if (rexBuilder.canRemoveCastFromLiteral(e.getType(), value, typeName)) {
+        return rexBuilder.makeCast(e.getType(), operand);
+      }
+
+      // Next, try to convert the value to a different type,
+      // e.g. CAST('123' as integer)
+      switch (literal.getTypeName()) {
+      case TIME:
+        switch (e.getType().getSqlTypeName()) {
+        case TIMESTAMP:
+          return e;
+        }
+      }
+      final List<RexNode> reducedValues = new ArrayList<>();
+      EXECUTOR.reduce(rexBuilder, ImmutableList.<RexNode>of(e), reducedValues);
+      return Preconditions.checkNotNull(
+          Iterables.getOnlyElement(reducedValues));
+    default:
+      return e;
+    }
+  }
+
   /** Returns a function that applies NOT to its argument. */
   public static Function<RexNode, RexNode> notFn(final RexBuilder rexBuilder) {
     return new Function<RexNode, RexNode>() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/main/java/org/apache/calcite/runtime/Hook.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/runtime/Hook.java b/core/src/main/java/org/apache/calcite/runtime/Hook.java
index 2d10b31..a97a406 100644
--- a/core/src/main/java/org/apache/calcite/runtime/Hook.java
+++ b/core/src/main/java/org/apache/calcite/runtime/Hook.java
@@ -16,6 +16,8 @@
  */
 package org.apache.calcite.runtime;
 
+import org.apache.calcite.util.Holder;
+
 import com.google.common.base.Function;
 
 import java.util.ArrayList;
@@ -33,6 +35,10 @@ public enum Hook {
    * in tests. */
   CURRENT_TIME,
 
+  /** Returns a boolean value, whether RelBuilder should simplify expressions.
+   * Default true. */
+  REL_BUILDER_SIMPLIFY,
+
   /** Called with the SQL string and parse tree, in an array. */
   PARSE_TREE,
 
@@ -129,6 +135,14 @@ public enum Hook {
     }
   }
 
+  /** Returns the value of a property hook.
+   * (Property hooks take a {@link Holder} as an argument.) */
+  public <V> V get(V defaultValue) {
+    final Holder<V> holder = Holder.of(defaultValue);
+    run(holder);
+    return holder.get();
+  }
+
   /** Removes a Hook after use.
    *
    * <p>Note: Although it would be convenient, this interface cannot extend

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
index 3742c8e..a502a01 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlCastFunction.java
@@ -38,10 +38,8 @@ import org.apache.calcite.sql.type.SqlTypeFamily;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 
-import com.google.common.collect.ImmutableSet;
-
-import java.util.Objects;
-import java.util.Set;
+import com.google.common.collect.ImmutableSetMultimap;
+import com.google.common.collect.SetMultimap;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -54,8 +52,22 @@ import static org.apache.calcite.util.Static.RESOURCE;
 public class SqlCastFunction extends SqlFunction {
   //~ Instance fields --------------------------------------------------------
 
-  private final Set<TypeFamilyCast> nonMonotonicPreservingCasts =
-      createNonMonotonicPreservingCasts();
+  /** Map of all casts that do not preserve monotonicity. */
+  private final SetMultimap<SqlTypeFamily, SqlTypeFamily> nonMonotonicCasts =
+      ImmutableSetMultimap.<SqlTypeFamily, SqlTypeFamily>builder()
+          .put(SqlTypeFamily.EXACT_NUMERIC, SqlTypeFamily.CHARACTER)
+          .put(SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER)
+          .put(SqlTypeFamily.APPROXIMATE_NUMERIC, SqlTypeFamily.CHARACTER)
+          .put(SqlTypeFamily.DATETIME_INTERVAL, SqlTypeFamily.CHARACTER)
+          .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.EXACT_NUMERIC)
+          .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.NUMERIC)
+          .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.APPROXIMATE_NUMERIC)
+          .put(SqlTypeFamily.CHARACTER, SqlTypeFamily.DATETIME_INTERVAL)
+          .put(SqlTypeFamily.DATETIME, SqlTypeFamily.TIME)
+          .put(SqlTypeFamily.TIMESTAMP, SqlTypeFamily.TIME)
+          .put(SqlTypeFamily.TIME, SqlTypeFamily.DATETIME)
+          .put(SqlTypeFamily.TIME, SqlTypeFamily.TIMESTAMP)
+          .build();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -71,38 +83,6 @@ public class SqlCastFunction extends SqlFunction {
 
   //~ Methods ----------------------------------------------------------------
 
-  /**
-   * List all casts that do not preserve monotonicity.
-   */
-  private Set<TypeFamilyCast> createNonMonotonicPreservingCasts() {
-    ImmutableSet.Builder<TypeFamilyCast> builder = ImmutableSet.builder();
-    add(builder, SqlTypeFamily.EXACT_NUMERIC, SqlTypeFamily.CHARACTER);
-    add(builder, SqlTypeFamily.NUMERIC, SqlTypeFamily.CHARACTER);
-    add(builder, SqlTypeFamily.APPROXIMATE_NUMERIC, SqlTypeFamily.CHARACTER);
-    add(builder, SqlTypeFamily.DATETIME_INTERVAL, SqlTypeFamily.CHARACTER);
-    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.EXACT_NUMERIC);
-    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.NUMERIC);
-    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.APPROXIMATE_NUMERIC);
-    add(builder, SqlTypeFamily.CHARACTER, SqlTypeFamily.DATETIME_INTERVAL);
-    add(builder, SqlTypeFamily.DATETIME, SqlTypeFamily.TIME);
-    add(builder, SqlTypeFamily.TIMESTAMP, SqlTypeFamily.TIME);
-    add(builder, SqlTypeFamily.TIME, SqlTypeFamily.DATETIME);
-    add(builder, SqlTypeFamily.TIME, SqlTypeFamily.TIMESTAMP);
-    return builder.build();
-  }
-
-  private void add(ImmutableSet.Builder<TypeFamilyCast> result,
-      SqlTypeFamily from, SqlTypeFamily to) {
-    result.add(new TypeFamilyCast(from, to));
-  }
-
-  private boolean isMonotonicPreservingCast(
-      RelDataTypeFamily castFrom,
-      RelDataTypeFamily castTo) {
-    return !nonMonotonicPreservingCasts.contains(
-        new TypeFamilyCast(castFrom, castTo));
-  }
-
   public RelDataType inferReturnType(
       SqlOperatorBinding opBinding) {
     assert opBinding.getOperandCount() == 2;
@@ -202,36 +182,12 @@ public class SqlCastFunction extends SqlFunction {
   @Override public SqlMonotonicity getMonotonicity(SqlOperatorBinding call) {
     RelDataTypeFamily castFrom = call.getOperandType(0).getFamily();
     RelDataTypeFamily castTo = call.getOperandType(1).getFamily();
-    if (isMonotonicPreservingCast(castFrom, castTo)) {
-      return call.getOperandMonotonicity(0);
-    } else {
+    if (castFrom instanceof SqlTypeFamily
+        && castTo instanceof SqlTypeFamily
+        && nonMonotonicCasts.containsEntry(castFrom, castTo)) {
       return SqlMonotonicity.NOT_MONOTONIC;
-    }
-  }
-
-  //~ Inner Classes ----------------------------------------------------------
-
-  /** Pair of source-target type families. */
-  private class TypeFamilyCast {
-    private final RelDataTypeFamily castFrom;
-    private final RelDataTypeFamily castTo;
-
-    public TypeFamilyCast(
-        RelDataTypeFamily castFrom,
-        RelDataTypeFamily castTo) {
-      this.castFrom = castFrom;
-      this.castTo = castTo;
-    }
-
-    @Override public boolean equals(Object o) {
-      return o == this
-          || o instanceof TypeFamilyCast
-          && castFrom.equals(((TypeFamilyCast) o).castFrom)
-          && castTo.equals(((TypeFamilyCast) o).castTo);
-    }
-
-    @Override public int hashCode() {
-      return Objects.hash(castFrom, castTo);
+    } else {
+      return call.getOperandMonotonicity(0);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
index 554e777..2973a2f 100644
--- a/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
+++ b/core/src/main/java/org/apache/calcite/tools/RelBuilder.java
@@ -45,6 +45,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.schema.SchemaPlus;
 import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.SemiJoinType;
@@ -123,6 +124,7 @@ public class RelBuilder {
   private final RelFactories.ValuesFactory valuesFactory;
   private final RelFactories.TableScanFactory scanFactory;
   private final Deque<Frame> stack = new ArrayDeque<>();
+  private final boolean simplify;
 
   protected RelBuilder(Context context, RelOptCluster cluster,
       RelOptSchema relOptSchema) {
@@ -131,6 +133,7 @@ public class RelBuilder {
     if (context == null) {
       context = Contexts.EMPTY_CONTEXT;
     }
+    this.simplify = Hook.REL_BUILDER_SIMPLIFY.get(true);
     this.aggregateFactory =
         Util.first(context.unwrap(RelFactories.AggregateFactory.class),
             RelFactories.DEFAULT_AGGREGATE_FACTORY);
@@ -825,12 +828,17 @@ public class RelBuilder {
       Iterable<String> fieldNames,
       boolean force) {
     final List<String> names = new ArrayList<>();
-    final List<RexNode> exprList = Lists.newArrayList(nodes);
+    final List<RexNode> exprList = new ArrayList<>();
+    for (RexNode node : nodes) {
+      if (simplify) {
+        node = RexUtil.simplifyPreservingType(getRexBuilder(), node);
+      }
+      exprList.add(node);
+    }
     final Iterator<String> nameIterator = fieldNames.iterator();
     for (RexNode node : nodes) {
       final String name = nameIterator.hasNext() ? nameIterator.next() : null;
-      final String name2 = inferAlias(exprList, node);
-      names.add(Util.first(name, name2));
+      names.add(name != null ? name : inferAlias(exprList, node));
     }
     final RelDataType inputRowType = peek().getRowType();
     if (!force && RexUtil.isIdentity(exprList, inputRowType)) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/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 3488e2a..0c3acc6 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
@@ -665,9 +665,6 @@ public abstract class SqlOperatorBaseTest {
             true);
       }
 
-      if (!enable) {
-        return;
-      }
       // Convert from string to type
       checkCastToScalarOkay(
           "'" + MAX_NUMERIC_STRINGS[i] + "'",
@@ -1138,6 +1135,22 @@ public abstract class SqlOperatorBaseTest {
     tester.checkNull("cast(null as boolean)");
   }
 
+  @Ignore("[CALCITE-1439] Handling errors during constant reduction")
+  @Test public void testCastInvalid() {
+    // Constant reduction kicks in and generates Java constants that throw
+    // when the class is loaded, thus ExceptionInInitializerError. We don't have
+    // a fix yet.
+    tester.checkScalarExact("cast('15' as integer)", "INTEGER NOT NULL", "15");
+    tester.checkFails("cast('15.4' as integer)", "xxx", true);
+    tester.checkFails("cast('15.6' as integer)", "xxx", true);
+    tester.checkFails("cast('ue' as boolean)", "xxx", true);
+    tester.checkFails("cast('' as boolean)", "xxx", true);
+    tester.checkFails("cast('' as integer)", "xxx", true);
+    tester.checkFails("cast('' as real)", "xxx", true);
+    tester.checkFails("cast('' as double)", "xxx", true);
+    tester.checkFails("cast('' as smallint)", "xxx", true);
+  }
+
   @Test public void testCastDateTime() {
     // Test cast for date/time/timestamp
     tester.setFor(SqlStdOperatorTable.CAST);
@@ -1261,6 +1274,15 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "cast('12:54:78' as TIME)", BAD_DATETIME_MESSAGE,
         true);
+    tester.checkFails(
+        "cast('12:34:5' as TIME)", BAD_DATETIME_MESSAGE,
+        true);
+    tester.checkFails(
+        "cast('12:3:45' as TIME)", BAD_DATETIME_MESSAGE,
+        true);
+    tester.checkFails(
+        "cast('1:23:45' as TIME)", BAD_DATETIME_MESSAGE,
+        true);
 
     // timestamp <-> string
     checkCastToString(
@@ -1311,6 +1333,9 @@ public abstract class SqlOperatorBaseTest {
     tester.checkFails(
         "cast('1945-01-24 25:42:25.34' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
         true);
+    tester.checkFails(
+        "cast('1945-1-24 12:23:34.454' as TIMESTAMP)", BAD_DATETIME_MESSAGE,
+        true);
 
     // date <-> string
     checkCastToString("DATE '1945-02-24'", null, "1945-02-24");
@@ -1321,6 +1346,10 @@ public abstract class SqlOperatorBaseTest {
         "1945-02-24",
         "DATE NOT NULL");
     tester.checkScalar(
+        "cast(' 1945-2-4 ' as DATE)",
+        "1945-02-04",
+        "DATE NOT NULL");
+    tester.checkScalar(
         "cast('  1945-02-24  ' as DATE)",
         "1945-02-24",
         "DATE NOT NULL");
@@ -1395,6 +1424,7 @@ public abstract class SqlOperatorBaseTest {
     tester.checkBoolean("cast('true' as boolean)", Boolean.TRUE);
     tester.checkBoolean("cast('false' as boolean)", Boolean.FALSE);
     tester.checkBoolean("cast('  trUe' as boolean)", Boolean.TRUE);
+    tester.checkBoolean("cast('  tr' || 'Ue' as boolean)", Boolean.TRUE);
     tester.checkBoolean("cast('  fALse' as boolean)", Boolean.FALSE);
     tester.checkFails(
         "cast('unknown' as boolean)", INVALID_CHAR_MESSAGE,

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/java/org/apache/calcite/test/QuidemTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/QuidemTest.java b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
index 2e8675c..a3dafba 100644
--- a/core/src/test/java/org/apache/calcite/test/QuidemTest.java
+++ b/core/src/test/java/org/apache/calcite/test/QuidemTest.java
@@ -67,10 +67,15 @@ public class QuidemTest {
     this.method = findMethod(path);
   }
 
-  /** Run a test from the command line. */
+  /** Runs a test from the command line.
+   *
+   * <p>For example:
+   *
+   * <blockquote><code>java QuidemTest sql/dummy.iq</code></blockquote> */
   public static void main(String[] args) throws Exception {
-    final String path = "sql/lateral.iq";
-    new QuidemTest(path).test();
+    for (String arg : args) {
+      new QuidemTest(arg).test();
+    }
   }
 
   private Method findMethod(String path) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
index 562286a..5a12335 100644
--- a/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelMetadataTest.java
@@ -1350,8 +1350,7 @@ public class RelMetadataTest extends SqlToRelTestBase {
     final RelMetadataQuery mq = RelMetadataQuery.instance();
     RelOptPredicateList list = mq.getPulledUpPredicates(rel);
     assertThat(list.pulledUpPredicates,
-        sortsAs("[<($0, 10), =($3, 'y'), =($4, CAST('1'):INTEGER NOT NULL), "
-            + "IS NULL($1), IS NULL($2)]"));
+        sortsAs("[<($0, 10), =($3, 'y'), =($4, 1), IS NULL($1), IS NULL($2)]"));
   }
 
   @Test public void testPullUpPredicatesOnNullableConstant() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
index 09b9110..221dca2 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptRulesTest.java
@@ -87,6 +87,7 @@ import org.apache.calcite.rel.rules.ValuesReduceRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeFactory;
 import org.apache.calcite.rex.RexNode;
+import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidator;
@@ -848,12 +849,15 @@ public class RelOptRulesTest extends RelOptTestBase {
 
     // There is "CAST(2 AS INTEGER)" in the plan because 2 has type "INTEGER NOT
     // NULL" and we need "INTEGER".
-    checkPlanning(program,
-        "select 1+2, d.deptno+(3+4), (5+6)+d.deptno, cast(null as integer),"
-            + " coalesce(2,null), row(7+8)"
-            + " from dept d inner join emp e"
-            + " on d.deptno = e.deptno + (5-5)"
-            + " where d.deptno=(7+8) and d.deptno=(8+7) and d.deptno=coalesce(2,null)");
+    final String sql = "select"
+        + " 1+2, d.deptno+(3+4), (5+6)+d.deptno, cast(null as integer),"
+        + " coalesce(2,null), row(7+8)"
+        + " from dept d inner join emp e"
+        + " on d.deptno = e.deptno + (5-5)"
+        + " where d.deptno=(7+8) and d.deptno=(8+7) and d.deptno=coalesce(2,null)";
+    sql(sql).with(program)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   /** Test case for
@@ -1241,7 +1245,7 @@ public class RelOptRulesTest extends RelOptTestBase {
             + " where a - b < 21");
   }
 
-  @Test public void testReduceCase() throws Exception {
+  @Ignore @Test public void testReduceCase() throws Exception {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
         .build();
@@ -1250,7 +1254,9 @@ public class RelOptRulesTest extends RelOptTestBase {
         + "  case when false then cast(2.1 as float)\n"
         + "   else cast(1 as integer) end as newcol\n"
         + "from emp";
-    checkPlanning(program, sql);
+    sql(sql).with(program)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   @Test public void testReduceConstantsIsNull() throws Exception {
@@ -1447,6 +1453,20 @@ public class RelOptRulesTest extends RelOptTestBase {
             + "where cast(e.job as varchar(1)) = 'Manager'");
   }
 
+  /** Tests that a cast from a TIME to a TIMESTAMP is not reduced. It is not
+   * constant because the result depends upon the current date. */
+  @Test public void testReduceCastTimeUnchanged() throws Exception {
+    HepProgram program = new HepProgramBuilder()
+        .addRuleInstance(ReduceExpressionsRule.PROJECT_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
+        .addRuleInstance(ReduceExpressionsRule.JOIN_INSTANCE)
+        .build();
+
+    sql("select cast(time '12:34:56' as timestamp) from emp as e")
+        .with(program)
+        .checkUnchanged();
+  }
+
   @Test public void testReduceCastAndConsts() throws Exception {
     HepProgram program = new HepProgramBuilder()
         .addRuleInstance(ReduceExpressionsRule.FILTER_INSTANCE)
@@ -2262,7 +2282,9 @@ public class RelOptRulesTest extends RelOptTestBase {
     final String sql = "select empno,\n"
         + "  deptno in (select deptno from sales.emp where empno < 20) as d\n"
         + "from sales.emp";
-    checkSubQuery(sql).check();
+    checkSubQuery(sql)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   @Test public void testExpandProjectInNullable() throws Exception {
@@ -2272,21 +2294,27 @@ public class RelOptRulesTest extends RelOptTestBase {
         + "select empno,\n"
         + "  deptno in (select deptno from e2 where empno < 20) as d\n"
         + "from e2";
-    checkSubQuery(sql).check();
+    checkSubQuery(sql)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   @Test public void testExpandProjectInComposite() throws Exception {
     final String sql = "select empno, (empno, deptno) in (\n"
         + "    select empno, deptno from sales.emp where empno < 20) as d\n"
         + "from sales.emp";
-    checkSubQuery(sql).check();
+    checkSubQuery(sql)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   @Test public void testExpandProjectExists() throws Exception {
     final String sql = "select empno,\n"
         + "  exists (select deptno from sales.emp where empno < 20) as d\n"
         + "from sales.emp";
-    checkSubQuery(sql).check();
+    checkSubQuery(sql)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   @Test public void testExpandFilterScalar() throws Exception {
@@ -2326,7 +2354,9 @@ public class RelOptRulesTest extends RelOptTestBase {
         + "   when false then 20\n"
         + "   else 30\n"
         + "   end";
-    checkSubQuery(sql).check();
+    checkSubQuery(sql)
+        .withProperty(Hook.REL_BUILDER_SIMPLIFY, false)
+        .check();
   }
 
   /** An EXISTS filter that can be converted into true/false. */

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
index 637ddef..1556094 100644
--- a/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/RelOptTestBase.java
@@ -27,10 +27,16 @@ import org.apache.calcite.rel.RelRoot;
 import org.apache.calcite.rel.metadata.ChainedRelMetadataProvider;
 import org.apache.calcite.rel.metadata.DefaultRelMetadataProvider;
 import org.apache.calcite.rel.metadata.RelMetadataProvider;
+import org.apache.calcite.runtime.Hook;
+import org.apache.calcite.util.Holder;
 
+import com.google.common.base.Function;
+import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Lists;
 
+import java.util.ArrayList;
 import java.util.List;
+import java.util.Map;
 
 import static org.hamcrest.CoreMatchers.is;
 import static org.hamcrest.CoreMatchers.notNullValue;
@@ -171,7 +177,7 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
 
   /** Sets the SQL statement for a test. */
   Sql sql(String sql) {
-    return new Sql(sql, null, true);
+    return new Sql(sql, null, true, ImmutableMap.<Hook, Function>of());
   }
 
   /** Allows fluent testing. */
@@ -179,27 +185,74 @@ abstract class RelOptTestBase extends SqlToRelTestBase {
     private final String sql;
     private final HepPlanner hepPlanner;
     private final boolean expand;
+    private final ImmutableMap<Hook, Function> hooks;
 
-    public Sql(String sql, HepPlanner hepPlanner, boolean expand) {
+    Sql(String sql, HepPlanner hepPlanner, boolean expand,
+        ImmutableMap<Hook, Function> hooks) {
       this.sql = sql;
       this.hepPlanner = hepPlanner;
       this.expand = expand;
+      this.hooks = hooks;
     }
 
     public Sql with(HepPlanner hepPlanner) {
-      return new Sql(sql, hepPlanner, expand);
+      return new Sql(sql, hepPlanner, expand, hooks);
     }
 
     public Sql with(HepProgram program) {
-      return new Sql(sql, new HepPlanner(program), expand);
+      return new Sql(sql, new HepPlanner(program), expand, hooks);
+    }
+
+    /** Adds a hook and a handler for that hook. Calcite will create a thread
+     * hook (by calling {@link Hook#addThread(com.google.common.base.Function)})
+     * just before running the query, and remove the hook afterwards. */
+    public <T> Sql withHook(Hook hook, Function<T, Void> handler) {
+      return new Sql(sql, hepPlanner, expand,
+          ImmutableMap.<Hook, Function>builder().putAll(hooks)
+              .put(hook, handler).build());
+    }
+
+    /** Returns a function that, when a hook is called, will "return" a given
+     * value. (Because of the way hooks work, it "returns" the value by writing
+     * into a {@link Holder}. */
+    private <V> Function<Holder<V>, Void> propertyHook(final V v) {
+      return new Function<Holder<V>, Void>() {
+        public Void apply(Holder<V> holder) {
+          holder.set(v);
+          return null;
+        }
+      };
+    }
+
+    public <V> Sql withProperty(Hook hook, V value) {
+      return withHook(hook, propertyHook(value));
     }
 
     public Sql expand(boolean expand) {
-      return new Sql(sql, hepPlanner, expand);
+      return new Sql(sql, hepPlanner, expand, hooks);
     }
 
     public void check() {
-      checkPlanning(tester.withExpand(expand), null, hepPlanner, sql);
+      check(false);
+    }
+
+    public void checkUnchanged() {
+      check(true);
+    }
+
+    private void check(boolean unchanged) {
+      final List<Hook.Closeable> closeables = new ArrayList<>();
+      try {
+        for (Map.Entry<Hook, Function> entry : hooks.entrySet()) {
+          closeables.add(entry.getKey().addThread(entry.getValue()));
+        }
+        checkPlanning(tester.withExpand(expand), null, hepPlanner, sql,
+            unchanged);
+      } finally {
+        for (Hook.Closeable closeable : closeables) {
+          closeable.close();
+        }
+      }
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
index 015ddda..9481d2f 100644
--- a/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
+++ b/core/src/test/java/org/apache/calcite/test/RexProgramTest.java
@@ -17,6 +17,8 @@
 package org.apache.calcite.test;
 
 import org.apache.calcite.adapter.java.JavaTypeFactory;
+import org.apache.calcite.avatica.util.ByteString;
+import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.jdbc.JavaTypeFactoryImpl;
 import org.apache.calcite.plan.Strong;
 import org.apache.calcite.rel.type.RelDataType;
@@ -34,19 +36,25 @@ import org.apache.calcite.rex.RexProgramBuilder;
 import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.type.SqlTypeAssignmentRules;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.TestUtil;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
+import com.google.common.collect.LinkedHashMultimap;
 import com.google.common.collect.Lists;
+import com.google.common.collect.Multimap;
 
 import org.junit.Before;
 import org.junit.Test;
 
 import java.math.BigDecimal;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Calendar;
 import java.util.List;
 
 import static org.hamcrest.CoreMatchers.equalTo;
@@ -1070,6 +1078,114 @@ public class RexProgramTest {
     assertThat(result.getType().getSqlTypeName(), is(SqlTypeName.BOOLEAN));
   }
 
+  @Test public void testSimplifyCastLiteral() {
+    final List<RexLiteral> literals = new ArrayList<>();
+    literals.add(
+        rexBuilder.makeExactLiteral(BigDecimal.ONE,
+            typeFactory.createSqlType(SqlTypeName.INTEGER)));
+    literals.add(
+        rexBuilder.makeExactLiteral(BigDecimal.valueOf(2),
+            typeFactory.createSqlType(SqlTypeName.BIGINT)));
+    literals.add(
+        rexBuilder.makeExactLiteral(BigDecimal.valueOf(3),
+            typeFactory.createSqlType(SqlTypeName.SMALLINT)));
+    literals.add(
+        rexBuilder.makeExactLiteral(BigDecimal.valueOf(4),
+            typeFactory.createSqlType(SqlTypeName.TINYINT)));
+    literals.add(
+        rexBuilder.makeExactLiteral(new BigDecimal("1234"),
+            typeFactory.createSqlType(SqlTypeName.DECIMAL, 4, 0)));
+    literals.add(
+        rexBuilder.makeExactLiteral(new BigDecimal("123.45"),
+            typeFactory.createSqlType(SqlTypeName.DECIMAL, 5, 2)));
+    literals.add(
+        rexBuilder.makeApproxLiteral(new BigDecimal("3.1415"),
+            typeFactory.createSqlType(SqlTypeName.REAL)));
+    literals.add(
+        rexBuilder.makeApproxLiteral(BigDecimal.valueOf(Math.E),
+            typeFactory.createSqlType(SqlTypeName.FLOAT)));
+    literals.add(
+        rexBuilder.makeApproxLiteral(BigDecimal.valueOf(Math.PI),
+            typeFactory.createSqlType(SqlTypeName.DOUBLE)));
+    literals.add(rexBuilder.makeLiteral(true));
+    literals.add(rexBuilder.makeLiteral(false));
+    literals.add(rexBuilder.makeLiteral("hello world"));
+    literals.add(rexBuilder.makeLiteral("1969-07-20 12:34:56"));
+    literals.add(rexBuilder.makeLiteral("1969-07-20"));
+    literals.add(rexBuilder.makeLiteral("12:34:45"));
+    literals.add((RexLiteral)
+        rexBuilder.makeLiteral(new ByteString(new byte[] {1, 2, -34, 0, -128}),
+            typeFactory.createSqlType(SqlTypeName.BINARY, 5), false));
+    literals.add(
+        rexBuilder.makeDateLiteral(cal(1974, Calendar.AUGUST, 9, 0, 0, 0)));
+    literals.add(rexBuilder.makeTimeLiteral(cal(0, 0, 0, 1, 23, 45), 0));
+    literals.add(
+        rexBuilder.makeTimestampLiteral(
+            cal(1974, Calendar.AUGUST, 9, 1, 23, 45), 0));
+
+    final Multimap<SqlTypeName, RexLiteral> map = LinkedHashMultimap.create();
+    for (RexLiteral literal : literals) {
+      map.put(literal.getTypeName(), literal);
+    }
+
+    final List<RelDataType> types = new ArrayList<>();
+    types.add(typeFactory.createSqlType(SqlTypeName.INTEGER));
+    types.add(typeFactory.createSqlType(SqlTypeName.BIGINT));
+    types.add(typeFactory.createSqlType(SqlTypeName.SMALLINT));
+    types.add(typeFactory.createSqlType(SqlTypeName.TINYINT));
+    types.add(typeFactory.createSqlType(SqlTypeName.REAL));
+    types.add(typeFactory.createSqlType(SqlTypeName.FLOAT));
+    types.add(typeFactory.createSqlType(SqlTypeName.DOUBLE));
+    types.add(typeFactory.createSqlType(SqlTypeName.BOOLEAN));
+    types.add(typeFactory.createSqlType(SqlTypeName.VARCHAR, 10));
+    types.add(typeFactory.createSqlType(SqlTypeName.CHAR, 5));
+    types.add(typeFactory.createSqlType(SqlTypeName.VARBINARY, 60));
+    types.add(typeFactory.createSqlType(SqlTypeName.BINARY, 3));
+    types.add(typeFactory.createSqlType(SqlTypeName.TIMESTAMP));
+    types.add(typeFactory.createSqlType(SqlTypeName.TIME));
+    types.add(typeFactory.createSqlType(SqlTypeName.DATE));
+
+    for (RelDataType fromType : types) {
+      for (RelDataType toType : types) {
+        if (SqlTypeAssignmentRules.instance().canCastFrom(
+            toType.getSqlTypeName(), fromType.getSqlTypeName(), false)) {
+          for (RexLiteral literal : map.get(fromType.getSqlTypeName())) {
+            final RexNode cast = rexBuilder.makeCast(toType, literal);
+            if (cast instanceof RexLiteral) {
+              assertThat(cast.getType(), is(toType));
+              continue; // makeCast already simplified
+            }
+            final RexNode simplified = RexUtil.simplify(rexBuilder, cast);
+            boolean expectedSimplify =
+                literal.getTypeName() != toType.getSqlTypeName()
+                || (literal.getTypeName() == SqlTypeName.CHAR
+                    && ((NlsString) literal.getValue()).getValue().length()
+                        > toType.getPrecision())
+                || (literal.getTypeName() == SqlTypeName.BINARY
+                    && ((ByteString) literal.getValue()).length()
+                        > toType.getPrecision());
+            boolean couldSimplify = !cast.equals(simplified);
+            final String reason = (expectedSimplify
+                ? "expected to simplify, but could not: "
+                : "simplified, but did not expect to: ")
+                + cast + " --> " + simplified;
+            assertThat(reason, couldSimplify, is(expectedSimplify));
+          }
+        }
+      }
+    }
+  }
+
+  private Calendar cal(int y, int m, int d, int h, int mm, int s) {
+    final Calendar c = Calendar.getInstance(DateTimeUtils.GMT_ZONE);
+    c.set(Calendar.YEAR, y);
+    c.set(Calendar.MONTH, m);
+    c.set(Calendar.DAY_OF_MONTH, d);
+    c.set(Calendar.HOUR_OF_DAY, h);
+    c.set(Calendar.MINUTE, mm);
+    c.set(Calendar.SECOND, s);
+    return c;
+  }
 }
 
 // End RexProgramTest.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
index 8edbd44..d830b95 100644
--- a/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/RelOptRulesTest.xml
@@ -2592,7 +2592,7 @@ LogicalAggregate(group=[{1}], EXPR$1=[$SUM0($2)])
             <![CDATA[
 LogicalAggregate(group=[{1}], EXPR$1=[$SUM0($2)])
   LogicalAggregate(group=[{5, 7}], EXPR$1=[COUNT()])
-    LogicalFilter(condition=[>(CAST($5):BIGINT NOT NULL, CAST('12'):BIGINT NOT NULL)])
+    LogicalFilter(condition=[>(CAST($5):BIGINT NOT NULL, 12)])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -3868,6 +3868,17 @@ LogicalProject(NEWCOL=[1E0])
 ]]>
         </Resource>
     </TestCase>
+    <TestCase name="testReduceCastTimeUnchanged">
+        <Resource name="sql">
+            <![CDATA[select cast(time '12:34:56' as timestamp) from emp as e]]>
+        </Resource>
+        <Resource name="planBefore">
+            <![CDATA[
+LogicalProject(EXPR$0=[CAST(12:34:56):TIMESTAMP(0) NOT NULL])
+  LogicalTableScan(table=[[CATALOG, SALES, EMP]])
+]]>
+        </Resource>
+    </TestCase>
     <TestCase name="testReduceConstantsRequiresExecutor">
         <Resource name="sql">
             <![CDATA[select * from (values (1,2)) where 1 + 2 > 3 + CAST(NULL AS INTEGER)]]>

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
----------------------------------------------------------------------
diff --git a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
index 92f6ed8..371c20c 100644
--- a/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
+++ b/core/src/test/resources/org/apache/calcite/test/SqlToRelConverterTest.xml
@@ -61,7 +61,9 @@ LogicalProject(EXPR$0=[+($0, 4)], EXPR$1=[$1], EXPR$2=[$2], EXPR$3=[*(2, $3)])
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select deptno + 4, sum(sal), sum(3 + sal), 2 * count(sal) from emp group by deptno]]>
+            <![CDATA[select
+  deptno + 4, sum(sal), sum(3 + sal), 2 * count(sal)
+from emp group by deptno]]>
         </Resource>
     </TestCase>
     <TestCase name="testHaving">
@@ -102,7 +104,10 @@ LogicalProject(NAME=[$1], FOO=[$2])
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select name, foo from (select deptno, name, count(deptno) as foo from dept group by name, deptno, name)]]>
+            <![CDATA[select name, foo from (
+select deptno, name, count(deptno) as foo
+from dept
+group by name, deptno, name)]]>
         </Resource>
     </TestCase>
     <TestCase name="testAggDistinct">
@@ -114,7 +119,9 @@ LogicalAggregate(group=[{0}], EXPR$1=[SUM($1)], EXPR$2=[SUM(DISTINCT $1)], EXPR$
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select deptno, sum(sal), sum(distinct sal), count(*) from emp group by deptno]]>
+            <![CDATA[select deptno, sum(sal), sum(distinct sal), count(*)
+from emp
+group by deptno]]>
         </Resource>
     </TestCase>
     <TestCase name="testCollectionTableWithLateral">
@@ -323,7 +330,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select*from emp where exists (select 1 from dept where deptno=55)]]>
+            <![CDATA[select*from emp
+where exists (select 1 from dept where deptno=55)]]>
         </Resource>
     </TestCase>
     <TestCase name="testExistsCorrelated">
@@ -341,7 +349,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select*from emp where exists (select 1 from dept where emp.deptno=dept.deptno)]]>
+            <![CDATA[select*from emp where exists (
+  select 1 from dept where emp.deptno=dept.deptno)]]>
         </Resource>
     </TestCase>
     <TestCase name="testUnnestSelect">
@@ -374,7 +383,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
 ]]>
         </Resource>
         <Resource name="sql">
-            <![CDATA[select * from emp, LATERAL (select * from dept where emp.deptno=dept.deptno)]]>
+            <![CDATA[select * from emp,
+  LATERAL (select * from dept where emp.deptno=dept.deptno)]]>
         </Resource>
     </TestCase>
     <TestCase name="testElement">
@@ -580,7 +590,8 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testIntervalLiteralYearToMonth">
         <Resource name="sql">
-            <![CDATA[select cast(empno as Integer) * (INTERVAL '1-1' YEAR TO MONTH)
+            <![CDATA[select
+  cast(empno as Integer) * (INTERVAL '1-1' YEAR TO MONTH)
 from emp]]>
         </Resource>
         <Resource name="plan">
@@ -592,7 +603,8 @@ LogicalProject(EXPR$0=[*(CAST($0):INTEGER NOT NULL, 13)])
     </TestCase>
     <TestCase name="testIntervalLiteralHourToMinute">
         <Resource name="sql">
-            <![CDATA[select cast(empno as Integer) * (INTERVAL '1:1' HOUR TO MINUTE)
+            <![CDATA[select
+ cast(empno as Integer) * (INTERVAL '1:1' HOUR TO MINUTE)
 from emp]]>
         </Resource>
         <Resource name="plan">
@@ -808,7 +820,8 @@ ProjectRel(EMPNO=[$0], Y=[$1])
     </TestCase>
     <TestCase name="testOrderByAliasInExpr">
         <Resource name="sql">
-            <![CDATA[select empno + 1 as x, empno - 2 as y from emp order by y + 3]]>
+            <![CDATA[select empno + 1 as x, empno - 2 as y
+from emp order by y + 3]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -821,7 +834,8 @@ LogicalProject(X=[$0], Y=[$1])
     </TestCase>
     <TestCase name="testOrderByAliasOverrides">
         <Resource name="sql">
-            <![CDATA[select empno + 1 as empno, empno - 2 as y from emp order by empno + 3]]>
+            <![CDATA[select empno + 1 as empno, empno - 2 as y
+from emp order by empno + 3]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -834,7 +848,10 @@ LogicalProject(EMPNO=[$0], Y=[$1])
     </TestCase>
     <TestCase name="testOrderUnion">
         <Resource name="sql">
-            <![CDATA[select empno, sal from emp union all select deptno, deptno from dept order by sal desc, empno asc]]>
+            <![CDATA[select empno, sal from emp
+union all
+select deptno, deptno from dept
+order by sal desc, empno asc]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -850,7 +867,10 @@ LogicalSort(sort0=[$1], sort1=[$0], dir0=[DESC], dir1=[ASC])
     </TestCase>
     <TestCase name="testOrderUnionExprs">
         <Resource name="sql">
-            <![CDATA[select empno, sal from emp union all select deptno, deptno from dept order by empno * sal + 2]]>
+            <![CDATA[select empno, sal from emp
+union all
+select deptno, deptno from dept
+order by empno * sal + 2]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -867,7 +887,10 @@ LogicalProject(EMPNO=[$0], SAL=[$1])
     </TestCase>
     <TestCase name="testOrderGroup">
         <Resource name="sql">
-            <![CDATA[select deptno, count(*) from emp group by deptno order by deptno * sum(sal) desc, min(empno)]]>
+            <![CDATA[select deptno, count(*)
+from emp
+group by deptno
+order by deptno * sum(sal) desc, min(empno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -882,7 +905,8 @@ LogicalProject(DEPTNO=[$0], EXPR$1=[$1])
     </TestCase>
     <TestCase name="testOrderDistinct">
         <Resource name="sql">
-            <![CDATA[select distinct empno, deptno + 1 from emp order by deptno + 1 + empno]]>
+            <![CDATA[select distinct empno, deptno + 1
+from emp order by deptno + 1 + empno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -896,7 +920,8 @@ LogicalProject(EMPNO=[$0], EXPR$1=[$1])
     </TestCase>
     <TestCase name="testOrderBySameExpr">
         <Resource name="sql">
-            <![CDATA[select empno from emp, dept order by sal + empno desc, sal * empno, sal + empno]]>
+            <![CDATA[select empno from emp, dept
+order by sal + empno desc, sal * empno, sal + empno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -911,7 +936,10 @@ LogicalProject(EMPNO=[$0])
     </TestCase>
     <TestCase name="testOrderUnionOrdinal">
         <Resource name="sql">
-            <![CDATA[select empno, sal from emp union all select deptno, deptno from dept order by 2]]>
+            <![CDATA[select empno, sal from emp
+union all
+select deptno, deptno from dept
+order by 2]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -971,7 +999,8 @@ LogicalProject(EXPR$0=[LAST_VALUE($7) OVER (ORDER BY $0 RANGE BETWEEN UNBOUNDED
     </TestCase>
     <TestCase name="testOverOrderFollowingWindow">
         <Resource name="sql">
-            <![CDATA[select last_value(deptno) over (order by empno rows 2 following)
+            <![CDATA[select
+  last_value(deptno) over (order by empno rows 2 following)
 from emp
 ]]>
         </Resource>
@@ -1037,7 +1066,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testJoinOn">
         <Resource name="sql">
-            <![CDATA[SELECT * FROM emp JOIN dept on emp.deptno = dept.deptno]]>
+            <![CDATA[SELECT * FROM emp
+JOIN dept on emp.deptno = dept.deptno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1197,7 +1227,9 @@ LogicalProject(A=[$0], B=[$1], C=[$2], DEPTNO=[$3], NAME=[$4])
     </TestCase>
     <TestCase name="testJoinWithUnion">
         <Resource name="sql">
-            <![CDATA[select grade from (select empno from emp union select deptno from dept), salgrade]]>
+            <![CDATA[select grade
+from (select empno from emp union select deptno from dept),
+  salgrade]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1227,7 +1259,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testJoinNaturalNoCommonColumn">
         <Resource name="sql">
-            <![CDATA[SELECT * FROM emp NATURAL JOIN (SELECT deptno AS foo, name FROM dept) AS d]]>
+            <![CDATA[SELECT *
+FROM emp NATURAL JOIN (SELECT deptno AS foo, name FROM dept) AS d]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1241,7 +1274,9 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testJoinNaturalMultipleCommonColumn">
         <Resource name="sql">
-            <![CDATA[SELECT * FROM emp NATURAL JOIN (SELECT deptno, name AS ename FROM dept) AS d]]>
+            <![CDATA[SELECT *
+FROM emp
+NATURAL JOIN (SELECT deptno, name AS ename FROM dept) AS d]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1255,7 +1290,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testOrderOffsetFetch">
         <Resource name="sql">
-            <![CDATA[select empno from emp order by empno offset 10 rows fetch next 5 rows only]]>
+            <![CDATA[select empno from emp
+order by empno offset 10 rows fetch next 5 rows only]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1279,7 +1315,8 @@ LogicalSort(fetch=[5])
     </TestCase>
     <TestCase name="testOffsetFetch">
         <Resource name="sql">
-            <![CDATA[select empno from emp offset 10 rows fetch next 5 rows only]]>
+            <![CDATA[select empno from emp
+offset 10 rows fetch next 5 rows only]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1336,7 +1373,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testJoinOnExpression">
         <Resource name="sql">
-            <![CDATA[SELECT * FROM emp JOIN dept on emp.deptno + 1 = dept.deptno - 2]]>
+            <![CDATA[SELECT * FROM emp
+JOIN dept on emp.deptno + 1 = dept.deptno - 2]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1379,7 +1417,9 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     <TestCase name="testWithUnion">
         <Resource name="sql">
             <![CDATA[with emp2 as (select * from emp where deptno > 10)
-select empno from emp2 where deptno < 30 union all select deptno from emp]]>
+select empno from emp2 where deptno < 30
+union all
+select deptno from emp]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1451,7 +1491,8 @@ LogicalAggregate(group=[{}], EXPR$0=[SUM($0)])
     </TestCase>
     <TestCase name="testConditionOffByOneReversed">
         <Resource name="sql">
-            <![CDATA[SELECT * FROM emp JOIN dept on dept.deptno = emp.deptno + 0]]>
+            <![CDATA[SELECT * FROM emp
+JOIN dept on dept.deptno = emp.deptno + 0]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1466,7 +1507,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testConditionOffByOne">
         <Resource name="sql">
-            <![CDATA[SELECT * FROM emp JOIN dept on emp.deptno + 0 = dept.deptno]]>
+            <![CDATA[SELECT * FROM emp
+JOIN dept on emp.deptno + 0 = dept.deptno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -1500,7 +1542,8 @@ LogicalProject(EMPNO=[$0], ENAME=[$1], JOB=[$2], MGR=[$3], HIREDATE=[$4], SAL=[$
     </TestCase>
     <TestCase name="testNestedCorrelations">
         <Resource name="sql">
-            <![CDATA[select * from (select 2+deptno d2, 3+deptno d3 from emp) e
+            <![CDATA[select *
+from (select 2+deptno d2, 3+deptno d3 from emp) e
  where exists (select 1 from (select deptno+1 d1 from dept) d
  where d1=e.d2 and exists (select 2 from (select deptno+4 d4, deptno+5 d5, deptno+6 d6 from dept)
  where d4=d.d1 and d5=d.d1 and d6=e.d3))]]>
@@ -1713,7 +1756,8 @@ LogicalProject(EXPR$0=[CAST(/(SUM(+(+($1, *(2, $2)), *(3, $3))) OVER (PARTITION
     </TestCase>
     <TestCase name="testWithAlias">
         <Resource name="sql">
-            <![CDATA[with w(x, y) as (select * from dept where deptno > 10)
+            <![CDATA[with w(x, y) as
+  (select * from dept where deptno > 10)
 select x from w where x < 30 union all select deptno from dept]]>
         </Resource>
         <Resource name="plan">
@@ -1774,18 +1818,18 @@ from dept]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
-LogicalProject(NAME=[$1], EXPR$1=[CASE(=($2, 0), false, IS NOT NULL($6), true, IS NULL($4), null, <($3, $2), null, false)])
+LogicalProject(NAME=[$1], EXPR$1=[CASE(=($2, 0), false, IS NOT NULL($6), true, <($3, $2), null, false)])
   LogicalJoin(condition=[=($4, $5)], joinType=[left])
     LogicalProject($f0=[$0], $f1=[$1], $f2=[$2], $f3=[$3], $f4=[$0])
       LogicalJoin(condition=[true], joinType=[inner])
         LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
         LogicalAggregate(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
           LogicalProject($f0=[$0], $f1=[true])
-            LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+            LogicalProject(EXPR$0=[CAST($7):INTEGER])
               LogicalTableScan(table=[[CATALOG, SALES, EMP]])
     LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
       LogicalProject($f0=[$0], $f1=[true])
-        LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+        LogicalProject(EXPR$0=[CAST($7):INTEGER])
           LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 ]]>
         </Resource>
@@ -1805,11 +1849,11 @@ LogicalProject(EMPNO=[$0], EXPR$1=[NOT(CASE(=($9, 0), false, IS NOT NULL($13), t
         LogicalTableScan(table=[[CATALOG, SALES, EMP]])
         LogicalAggregate(group=[{}], agg#0=[COUNT()], agg#1=[COUNT($0)])
           LogicalProject($f0=[$0], $f1=[true])
-            LogicalProject(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
+            LogicalProject(EXPR$0=[CAST($0):INTEGER])
               LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
     LogicalAggregate(group=[{0}], agg#0=[MIN($1)])
       LogicalProject($f0=[$0], $f1=[true])
-        LogicalProject(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
+        LogicalProject(EXPR$0=[CAST($0):INTEGER])
           LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 ]]>
         </Resource>
@@ -1958,7 +2002,8 @@ LogicalProject(EXPR$0=[$6])
     </TestCase>
     <TestCase name="testGroupingSetsCartesianProduct">
         <Resource name="sql">
-            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+            <![CDATA[select 1
+from (values (1, 2, 3, 4)) as t(a, b, c, d)
 group by grouping sets (a, b), grouping sets (c, d)]]>
         </Resource>
         <Resource name="plan">
@@ -1972,7 +2017,8 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testGroupingSetsCartesianProduct2">
         <Resource name="sql">
-            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+            <![CDATA[select 1
+from (values (1, 2, 3, 4)) as t(a, b, c, d)
 group by grouping sets (a, (a, b)), grouping sets (c), d]]>
         </Resource>
         <Resource name="plan">
@@ -1986,7 +2032,8 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testRollup">
         <Resource name="sql">
-            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+            <![CDATA[select 1
+from (values (1, 2, 3, 4)) as t(a, b, c, d)
 group by rollup(a, b), rollup(c, d)]]>
         </Resource>
         <Resource name="plan">
@@ -2000,7 +2047,8 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testCube">
         <Resource name="sql">
-            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+            <![CDATA[select 1
+from (values (1, 2, 3, 4)) as t(a, b, c, d)
 group by cube(a, b)]]>
         </Resource>
         <Resource name="plan">
@@ -2015,7 +2063,8 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testRollupTuples">
         <Resource name="sql">
-            <![CDATA[select 1 from (values (1, 2, 3, 4)) as t(a, b, c, d)
+            <![CDATA[select 1
+from (values (1, 2, 3, 4)) as t(a, b, c, d)
 group by rollup(b, (a, d))]]>
         </Resource>
         <Resource name="plan">
@@ -2061,7 +2110,9 @@ LogicalProject(A=[$0], B=[$1], C=[$4])
     </TestCase>
     <TestCase name="testGroupByExpression">
         <Resource name="sql">
-            <![CDATA[select count(*) from emp group by substring(ename FROM 1 FOR 1)]]>
+            <![CDATA[select count(*)
+from emp
+group by substring(ename FROM 1 FOR 1)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -2089,7 +2140,8 @@ LogicalProject(EXPR$0=[1])
     </TestCase>
     <TestCase name="testGroupingFunctionWithGroupBy">
         <Resource name="sql">
-            <![CDATA[select deptno, grouping(deptno), count(*), grouping(empno)
+            <![CDATA[select
+  deptno, grouping(deptno), count(*), grouping(empno)
 from emp
 group by empno, deptno
 order by 2]]>
@@ -2106,7 +2158,8 @@ LogicalSort(sort0=[$1], dir0=[ASC])
     </TestCase>
     <TestCase name="testGroupingFunction">
         <Resource name="sql">
-            <![CDATA[select deptno, grouping(deptno), count(*), grouping(empno)
+            <![CDATA[select
+  deptno, grouping(deptno), count(*), grouping(empno)
 from emp
 group by rollup(empno, deptno)]]>
         </Resource>
@@ -2147,8 +2200,10 @@ LogicalAggregate(group=[{0}])
     </TestCase>
     <TestCase name="testAggNoDuplicateColumnNames">
         <Resource name="sql">
-            <![CDATA[SELECT empno, EXPR$2, COUNT(empno) FROM (SELECT empno, deptno AS EXPR$2
-FROM emp) GROUP BY empno, EXPR$2]]>
+            <![CDATA[SELECT  empno, EXPR$2, COUNT(empno) FROM (
+    SELECT empno, deptno AS EXPR$2
+    FROM emp)
+GROUP BY empno, EXPR$2]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -2218,7 +2273,8 @@ LogicalDelta
     </TestCase>
     <TestCase name="testStreamGroupBy">
         <Resource name="sql">
-            <![CDATA[select stream floor(rowtime to second) as rowtime, count(*) as c
+            <![CDATA[select stream
+ floor(rowtime to second) as rowtime, count(*) as c
 from orders
 group by floor(rowtime to second)]]>
         </Resource>
@@ -2249,7 +2305,8 @@ LogicalDelta
     </TestCase>
     <TestCase name="testGroupByCaseIn">
         <Resource name="sql">
-            <![CDATA[select (CASE WHEN (deptno IN (10, 20)) THEN 0 ELSE deptno END),
+            <![CDATA[select
+ (CASE WHEN (deptno IN (10, 20)) THEN 0 ELSE deptno END),
  min(empno) from EMP
 group by (CASE WHEN (deptno IN (10, 20)) THEN 0 ELSE deptno END)]]>
         </Resource>
@@ -2263,7 +2320,10 @@ LogicalAggregate(group=[{0}], EXPR$1=[MIN($1)])
     </TestCase>
     <TestCase name="testAggFilter">
         <Resource name="sql">
-            <![CDATA[select deptno, sum(sal * 2) filter (where empno < 10), count(*) from emp group by deptno]]>
+            <![CDATA[select
+  deptno, sum(sal * 2) filter (where empno < 10), count(*)
+from emp
+group by deptno]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -2614,7 +2674,8 @@ LogicalProject(EXPR$0=[$2], EXPR$1=[RANK() OVER (ORDER BY $1 RANGE BETWEEN UNBOU
     </TestCase>
     <TestCase name="testWindowAggInSubQueryJoin">
         <Resource name="sql">
-            <![CDATA[select T.x, T.y, T.z, emp.empno from (select min(deptno) as x,
+            <![CDATA[select T.x, T.y, T.z, emp.empno
+from (select min(deptno) as x,
    rank() over (order by empno) as y,
    max(empno) over (partition by deptno) as z
    from emp group by deptno, empno) as T
@@ -2637,8 +2698,7 @@ LogicalProject(X=[$0], Y=[$1], Z=[$2], EMPNO=[$3])
     <TestCase name="testOrderByOver">
         <Resource name="sql">
             <![CDATA[select deptno, rank() over(partition by empno order by deptno)
-   from emp order by row_number() over(partition by empno order by deptno)
-]]>
+from emp order by row_number() over(partition by empno order by deptno)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -2736,7 +2796,8 @@ LogicalProject(EMPNO=[$0])
     </TestCase>
     <TestCase name="testCorrelationNotExistsAndFilter">
         <Resource name="sql">
-            <![CDATA[SELECT e1.empno FROM emp e1, dept d1 where e1.deptno = d1.deptno
+            <![CDATA[SELECT e1.empno
+FROM emp e1, dept d1 where e1.deptno = d1.deptno
 and e1.deptno < 10 and d1.deptno < 15
 and not exists (select * from emp e2 where e1.empno = e2.empno)]]>
         </Resource>
@@ -2922,7 +2983,7 @@ from dept]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(NAME=[$1], EXPR$1=[IN($0, {
-LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+LogicalProject(EXPR$0=[CAST($7):INTEGER])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 })])
   LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
@@ -2969,7 +3030,7 @@ from emp]]>
         <Resource name="plan">
             <![CDATA[
 LogicalProject(EMPNO=[$0], EXPR$1=[NOT(IN($7, {
-LogicalProject(EXPR$0=[CASE(true, CAST($0):INTEGER, null)])
+LogicalProject(EXPR$0=[CAST($0):INTEGER])
   LogicalTableScan(table=[[CATALOG, SALES, DEPT]])
 }))])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -3076,7 +3137,7 @@ and deptno in (
             <![CDATA[
 LogicalProject(S=[$1])
   LogicalFilter(condition=[AND(>($2, 2), IN($0, {
-LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+LogicalProject(EXPR$0=[CAST($7):INTEGER])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 }))])
     LogicalAggregate(group=[{0}], S=[SUM($1)], agg#1=[COUNT()])
@@ -3098,7 +3159,7 @@ order by (select case when true then deptno else null end from emp) desc,
 LogicalProject(S=[$0])
   LogicalSort(sort0=[$1], sort1=[$2], dir0=[DESC], dir1=[ASC])
     LogicalProject(S=[$1], EXPR$1=[$SCALAR_QUERY({
-LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+LogicalProject(EXPR$0=[CAST($7):INTEGER])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 })], EXPR$2=[$2])
       LogicalAggregate(group=[{0}], S=[SUM($1)], agg#1=[COUNT()])
@@ -3119,7 +3180,7 @@ order by (select case when true then deptno else null end from emp) desc,
 LogicalProject(ENAME=[$0])
   LogicalSort(sort0=[$1], sort1=[$0], dir0=[DESC], dir1=[ASC])
     LogicalProject(ENAME=[$1], EXPR$1=[$SCALAR_QUERY({
-LogicalProject(EXPR$0=[CASE(true, CAST($7):INTEGER, null)])
+LogicalProject(EXPR$0=[CAST($7):INTEGER])
   LogicalTableScan(table=[[CATALOG, SALES, EMP]])
 })])
       LogicalTableScan(table=[[CATALOG, SALES, EMP]])
@@ -3545,7 +3606,8 @@ LogicalAggregate(group=[{0}])
     </TestCase>
     <TestCase name="testDynStarInExistSubQ">
         <Resource name="sql">
-            <![CDATA[select * from SALES.REGION where exists (select * from SALES.NATION)]]>
+            <![CDATA[select *
+from SALES.REGION where exists (select * from SALES.NATION)]]>
         </Resource>
         <Resource name="plan">
             <![CDATA[
@@ -3575,7 +3637,7 @@ LogicalProject(**=[$0])
     </TestCase>
     <TestCase name="testInToSemiJoin">
         <Resource name="sql">
-            <![CDATA[SELECT * from SALES.NATION order by n_nationkey]]>
+            <![CDATA[SELECT empno FROM emp AS e WHERE cast(e.empno as bigint) in (130, 131, 132, 133, 134)]]>
         </Resource>
         <Resource name="planNotConverted">
             <![CDATA[

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/resources/sql/dummy.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/dummy.iq b/core/src/test/resources/sql/dummy.iq
index 166e1b0..a70271c 100644
--- a/core/src/test/resources/sql/dummy.iq
+++ b/core/src/test/resources/sql/dummy.iq
@@ -16,8 +16,17 @@
 # limitations under the License.
 #
 !use post
-values 1;
-EXPR$0
-1
+!set outputformat mysql
+values cast('-123.45' as double);
++---------+
+| EXPR$0  |
++---------+
+| -123.45 |
++---------+
+(1 row)
+
 !ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[-1.2345E2], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
 # End dummy.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/core/src/test/resources/sql/misc.iq
----------------------------------------------------------------------
diff --git a/core/src/test/resources/sql/misc.iq b/core/src/test/resources/sql/misc.iq
index 1509a3c..068736b 100644
--- a/core/src/test/resources/sql/misc.iq
+++ b/core/src/test/resources/sql/misc.iq
@@ -1310,4 +1310,224 @@ select (case when (true) then 1 end) from (values(1));
 EXPR$0 INTEGER(10)
 !type
 
+# Cast an character literal to a timestamp; note: the plan does not contain CAST
+values cast('1969-07-21 12:34:56' as timestamp);
++---------------------+
+| EXPR$0              |
++---------------------+
+| 1969-07-21 12:34:56 |
++---------------------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[1969-07-21 12:34:56], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Cast a character literal to a date; note: the plan does not contain CAST
+values cast('1969-07-21' as date);
++------------+
+| EXPR$0     |
++------------+
+| 1969-07-21 |
++------------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[1969-07-21], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Slightly different format
+# (Incidentally, this format is not allowed in date literals, per the standard)
+values cast('1989-7-4' as date);
++------------+
+| EXPR$0     |
++------------+
+| 1989-07-04 |
++------------+
+(1 row)
+
+!ok
+
+# Cast a character literal to an integer; note: the plan does not contain CAST
+values cast('196907' as integer);
++--------+
+| EXPR$0 |
++--------+
+| 196907 |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[196907], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Cast an integer literal to a bigint; note: the plan does not contain CAST
+values cast(123 as bigint);
++--------+
+| EXPR$0 |
++--------+
+|    123 |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[123], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Cast an integer literal to a decimal; note: the plan does not contain CAST
+values cast('123.45' as decimal(4, 2));
++--------+
+| EXPR$0 |
++--------+
+| 123.45 |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[123.45], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Cast a character literal to a decimal; note: the plan does not contain CAST
+values cast('123.45' as decimal(4, 2));
++--------+
+| EXPR$0 |
++--------+
+| 123.45 |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[123.45], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+# Cast a character literal to a double; note: the plan does not contain CAST
+values cast('-123.45' as double);
++---------+
+| EXPR$0  |
++---------+
+| -123.45 |
++---------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[-1.2345E2], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+values cast('false' as boolean);
++--------+
+| EXPR$0 |
++--------+
+| false  |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[false], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+values cast('TRUE' as boolean);
++--------+
+| EXPR$0 |
++--------+
+| true   |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=[true], EXPR$0=[$t1])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+values cast('TR' || 'UE' as boolean);
++--------+
+| EXPR$0 |
++--------+
+| true   |
++--------+
+(1 row)
+
+!ok
+EnumerableCalc(expr#0=[{inputs}], expr#1=['TR'], expr#2=['UE'], expr#3=[||($t1, $t2)], expr#4=[CAST($t3):BOOLEAN NOT NULL], EXPR$0=[$t4])
+  EnumerableValues(tuples=[[{ 0 }]])
+!plan
+
+!if (fixed.calcite1439) {
+values cast('null' as boolean);
+!error
+!plan
+
+values cast('' as date);
+!error
+
+values cast('' as timestamp);
+!error
+
+values cast('' as integer);
+!error
+
+values cast('' as boolean);
+!error
+
+values cast('' as double);
+!error
+
+# Postgres fails:
+#  ERROR:  invalid input syntax for integer: "1.56"
+values cast('15.6' as integer);
+!error
+
+# Postgres fails:
+#  ERROR:  invalid input syntax for integer: " - 5 "
+values cast(' - 5 ' as double);
+!error
+
+# Out of TINYINT range (max 127)
+values cast('200' as tinyint);
+!error
+
+# Out of SMALLINT range (max 32767)
+values cast('50000' as smallint);
+!error
+
+# Out of INTEGER range (max 2.1e9)
+values cast('4567891234' as integer);
+!error
+
+# Out of BIGINT range (max 9.2e18)
+values cast('12345678901234567890' as bigint);
+!error
+
+# Out of REAL range
+values cast('12.34e56' as real);
+!error
+
+# Out of FLOAT range
+values cast('12.34e5678' as float);
+!error
+
+# Out of DOUBLE range
+values cast('12.34e5678' as double);
+!error
+
+!}
+
+# Postgres succeeds
+values cast(' -5 ' as double);
++--------+
+| EXPR$0 |
++--------+
+|   -5.0 |
++--------+
+(1 row)
+
+!ok
+
 # End misc.iq

http://git-wip-us.apache.org/repos/asf/calcite/blob/54556b82/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
----------------------------------------------------------------------
diff --git a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
index 65ade8e..4c81b60 100644
--- a/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
+++ b/druid/src/main/java/org/apache/calcite/adapter/druid/DruidDateTimeUtils.java
@@ -16,7 +16,6 @@
  */
 package org.apache.calcite.adapter.druid;
 
-import org.apache.calcite.avatica.util.DateTimeUtils;
 import org.apache.calcite.avatica.util.TimeUnitRange;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexCall;
@@ -24,6 +23,7 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexLiteral;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.sql.SqlKind;
+import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
 import com.google.common.base.Function;
@@ -38,10 +38,7 @@ import org.joda.time.chrono.ISOChronology;
 
 import org.slf4j.Logger;
 
-import java.sql.Date;
-import java.sql.Timestamp;
 import java.util.ArrayList;
-import java.util.Arrays;
 import java.util.Calendar;
 import java.util.List;
 import java.util.regex.Pattern;
@@ -67,7 +64,7 @@ public class DruidDateTimeUtils {
    * reference a single column: the timestamp column.
    */
   public static List<Interval> createInterval(RelDataType type, RexNode e) {
-    final List<Range> ranges = extractRanges(type, e, false);
+    final List<Range<Calendar>> ranges = extractRanges(e, false);
     if (ranges == null) {
       // We did not succeed, bail out
       return null;
@@ -82,32 +79,37 @@ public class DruidDateTimeUtils {
     return toInterval(ImmutableList.<Range>copyOf(condensedRanges.asRanges()));
   }
 
-  protected static List<Interval> toInterval(List<Range> ranges) {
-    List<Interval> intervals = Lists.transform(ranges, new Function<Range, Interval>() {
-      @Override public Interval apply(Range range) {
-        if (!range.hasLowerBound() && !range.hasUpperBound()) {
-          return DruidTable.DEFAULT_INTERVAL;
-        }
-        long start = range.hasLowerBound() ? toLong(range.lowerEndpoint())
-            : DruidTable.DEFAULT_INTERVAL.getStartMillis();
-        long end = range.hasUpperBound() ? toLong(range.upperEndpoint())
-            : DruidTable.DEFAULT_INTERVAL.getEndMillis();
-        if (range.hasLowerBound() && range.lowerBoundType() == BoundType.OPEN) {
-          start++;
-        }
-        if (range.hasUpperBound() && range.upperBoundType() == BoundType.CLOSED) {
-          end++;
-        }
-        return new Interval(start, end, ISOChronology.getInstanceUTC());
-      }
-    });
+  protected static List<Interval> toInterval(List<Range<Calendar>> ranges) {
+    List<Interval> intervals = Lists.transform(ranges,
+        new Function<Range<Calendar>, Interval>() {
+          @Override public Interval apply(Range<Calendar> range) {
+            if (!range.hasLowerBound() && !range.hasUpperBound()) {
+              return DruidTable.DEFAULT_INTERVAL;
+            }
+            long start = range.hasLowerBound()
+                ? range.lowerEndpoint().getTime().getTime()
+                : DruidTable.DEFAULT_INTERVAL.getStartMillis();
+            long end = range.hasUpperBound()
+                ? range.upperEndpoint().getTime().getTime()
+                : DruidTable.DEFAULT_INTERVAL.getEndMillis();
+            if (range.hasLowerBound()
+                && range.lowerBoundType() == BoundType.OPEN) {
+              start++;
+            }
+            if (range.hasUpperBound()
+                && range.upperBoundType() == BoundType.CLOSED) {
+              end++;
+            }
+            return new Interval(start, end, ISOChronology.getInstanceUTC());
+          }
+        });
     if (LOGGER.isInfoEnabled()) {
       LOGGER.info("Converted time ranges " + ranges + " to interval " + intervals);
     }
     return intervals;
   }
 
-  protected static List<Range> extractRanges(RelDataType type, RexNode node,
+  protected static List<Range<Calendar>> extractRanges(RexNode node,
       boolean withNot) {
     switch (node.getKind()) {
     case EQUALS:
@@ -117,16 +119,16 @@ public class DruidDateTimeUtils {
     case GREATER_THAN_OR_EQUAL:
     case BETWEEN:
     case IN:
-      return leafToRanges(type, (RexCall) node, withNot);
+      return leafToRanges((RexCall) node, withNot);
 
     case NOT:
-      return extractRanges(type, ((RexCall) node).getOperands().get(0), !withNot);
+      return extractRanges(((RexCall) node).getOperands().get(0), !withNot);
 
     case OR: {
       RexCall call = (RexCall) node;
-      List<Range> intervals = Lists.newArrayList();
+      List<Range<Calendar>> intervals = Lists.newArrayList();
       for (RexNode child : call.getOperands()) {
-        List<Range> extracted = extractRanges(type, child, withNot);
+        List<Range<Calendar>> extracted = extractRanges(child, withNot);
         if (extracted != null) {
           intervals.addAll(extracted);
         }
@@ -136,9 +138,9 @@ public class DruidDateTimeUtils {
 
     case AND: {
       RexCall call = (RexCall) node;
-      List<Range> ranges = new ArrayList<>();
+      List<Range<Calendar>> ranges = new ArrayList<>();
       for (RexNode child : call.getOperands()) {
-        List<Range> extractedRanges = extractRanges(type, child, false);
+        List<Range<Calendar>> extractedRanges = extractRanges(child, false);
         if (extractedRanges == null || extractedRanges.isEmpty()) {
           // We could not extract, we bail out
           return null;
@@ -147,7 +149,7 @@ public class DruidDateTimeUtils {
           ranges.addAll(extractedRanges);
           continue;
         }
-        List<Range> overlapped = Lists.newArrayList();
+        List<Range<Calendar>> overlapped = new ArrayList<>();
         for (Range current : ranges) {
           for (Range interval : extractedRanges) {
             if (current.isConnected(interval)) {
@@ -165,7 +167,7 @@ public class DruidDateTimeUtils {
     }
   }
 
-  protected static List<Range> leafToRanges(RelDataType type, RexCall call,
+  protected static List<Range<Calendar>> leafToRanges(RexCall call,
       boolean withNot) {
     switch (call.getKind()) {
     case EQUALS:
@@ -174,246 +176,81 @@ public class DruidDateTimeUtils {
     case GREATER_THAN:
     case GREATER_THAN_OR_EQUAL:
     {
-      RexLiteral literal = null;
+      final Calendar value;
       if (call.getOperands().get(0) instanceof RexInputRef
-          && call.getOperands().get(1) instanceof RexLiteral) {
-        literal = extractLiteral(call.getOperands().get(1));
-      } else if (call.getOperands().get(0) instanceof RexInputRef
-          && call.getOperands().get(1).getKind() == SqlKind.CAST) {
-        literal = extractLiteral(call.getOperands().get(1));
-      } else if (call.getOperands().get(1) instanceof RexInputRef
-          && call.getOperands().get(0) instanceof RexLiteral) {
-        literal = extractLiteral(call.getOperands().get(0));
+          && literalValue(call.getOperands().get(1)) != null) {
+        value = literalValue(call.getOperands().get(1));
       } else if (call.getOperands().get(1) instanceof RexInputRef
-          && call.getOperands().get(0).getKind() == SqlKind.CAST) {
-        literal = extractLiteral(call.getOperands().get(0));
-      }
-      if (literal == null) {
-        return null;
-      }
-      Comparable value = literalToType(literal, type);
-      if (value == null) {
+          && literalValue(call.getOperands().get(0)) != null) {
+        value = literalValue(call.getOperands().get(0));
+      } else {
         return null;
       }
       switch (call.getKind()) {
       case LESS_THAN:
-        return Arrays.<Range>asList(withNot ? Range.atLeast(value) : Range.lessThan(value));
+        return ImmutableList.of(withNot ? Range.atLeast(value) : Range.lessThan(value));
       case LESS_THAN_OR_EQUAL:
-        return Arrays.<Range>asList(withNot ? Range.greaterThan(value) : Range.atMost(value));
+        return ImmutableList.of(withNot ? Range.greaterThan(value) : Range.atMost(value));
       case GREATER_THAN:
-        return Arrays.<Range>asList(withNot ? Range.atMost(value) : Range.greaterThan(value));
+        return ImmutableList.of(withNot ? Range.atMost(value) : Range.greaterThan(value));
       case GREATER_THAN_OR_EQUAL:
-        return Arrays.<Range>asList(withNot ? Range.lessThan(value) : Range.atLeast(value));
+        return ImmutableList.of(withNot ? Range.lessThan(value) : Range.atLeast(value));
       default:
         if (!withNot) {
-          return Arrays.<Range>asList(Range.closed(value, value));
+          return ImmutableList.of(Range.closed(value, value));
         }
-        return Arrays.<Range>asList(Range.lessThan(value), Range.greaterThan(value));
+        return ImmutableList.of(Range.lessThan(value), Range.greaterThan(value));
       }
     }
     case BETWEEN:
     {
-      RexLiteral literal1 = extractLiteral(call.getOperands().get(2));
-      if (literal1 == null) {
-        return null;
-      }
-      RexLiteral literal2 = extractLiteral(call.getOperands().get(3));
-      if (literal2 == null) {
-        return null;
-      }
-      Comparable value1 = literalToType(literal1, type);
-      Comparable value2 = literalToType(literal2, type);
-      if (value1 == null || value2 == null) {
+      final Calendar value1;
+      final Calendar value2;
+      if (literalValue(call.getOperands().get(2)) != null
+          && literalValue(call.getOperands().get(3)) != null) {
+        value1 = literalValue(call.getOperands().get(2));
+        value2 = literalValue(call.getOperands().get(3));
+      } else {
         return null;
       }
+
       boolean inverted = value1.compareTo(value2) > 0;
       if (!withNot) {
-        return Arrays.<Range>asList(
+        return ImmutableList.of(
             inverted ? Range.closed(value2, value1) : Range.closed(value1, value2));
       }
-      return Arrays.<Range>asList(Range.lessThan(inverted ? value2 : value1),
+      return ImmutableList.of(Range.lessThan(inverted ? value2 : value1),
           Range.greaterThan(inverted ? value1 : value2));
     }
     case IN:
     {
-      List<Range> ranges = Lists.newArrayList();
-      for (int i = 1; i < call.getOperands().size(); i++) {
-        RexLiteral literal = extractLiteral(call.getOperands().get(i));
-        if (literal == null) {
-          return null;
-        }
-        Comparable element = literalToType(literal, type);
+      ImmutableList.Builder<Range<Calendar>> ranges = ImmutableList.builder();
+      for (RexNode operand : Util.skip(call.operands)) {
+        final Calendar element = literalValue(operand);
         if (element == null) {
           return null;
         }
         if (withNot) {
-          ranges.addAll(
-              Arrays.<Range>asList(Range.lessThan(element), Range.greaterThan(element)));
+          ranges.add(Range.lessThan(element));
+          ranges.add(Range.greaterThan(element));
         } else {
           ranges.add(Range.closed(element, element));
         }
       }
-      return ranges;
+      return ranges.build();
     }
     default:
       return null;
     }
   }
 
-  @SuppressWarnings("incomplete-switch")
-  protected static Comparable literalToType(RexLiteral literal, RelDataType type) {
-    // Extract
-    Object value = null;
-    switch (literal.getType().getSqlTypeName()) {
-    case DATE:
-    case TIME:
-    case TIMESTAMP:
-    case INTERVAL_YEAR_MONTH:
-    case INTERVAL_DAY:
-      value = literal.getValue();
-      break;
-    case TINYINT:
-    case SMALLINT:
-    case INTEGER:
-    case BIGINT:
-    case DOUBLE:
-    case DECIMAL:
-    case FLOAT:
-    case REAL:
-    case VARCHAR:
-    case CHAR:
-    case BOOLEAN:
-      value = literal.getValue3();
-    }
-    if (value == null) {
-      return null;
-    }
-
-    // Convert
-    switch (type.getSqlTypeName()) {
-    case BIGINT:
-      return toLong(value);
-    case INTEGER:
-      return toInt(value);
-    case FLOAT:
-      return toFloat(value);
-    case DOUBLE:
-      return toDouble(value);
-    case VARCHAR:
-    case CHAR:
-      return String.valueOf(value);
-    case TIMESTAMP:
-      return toTimestamp(value);
-    }
-    return null;
-  }
-
-  private static RexLiteral extractLiteral(RexNode node) {
-    RexNode target = node;
-    if (node.getKind() == SqlKind.CAST) {
-      target = ((RexCall) node).getOperands().get(0);
-    }
-    if (!(target instanceof RexLiteral)) {
-      return null;
-    }
-    return (RexLiteral) target;
-  }
-
-  private static Comparable toTimestamp(Object literal) {
-    if (literal instanceof Timestamp) {
-      return (Timestamp) literal;
-    }
-    final Long v = toLong(literal);
-    if (v != null) {
-      return new Timestamp(v);
-    }
-    return null;
-  }
-
-  private static Long toLong(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).longValue();
-    }
-    if (literal instanceof Date) {
-      return ((Date) literal).getTime();
-    }
-    if (literal instanceof Timestamp) {
-      return ((Timestamp) literal).getTime();
-    }
-    if (literal instanceof Calendar) {
-      return ((Calendar) literal).getTime().getTime();
-    }
-    if (literal instanceof String) {
-      final String s = (String) literal;
-      try {
-        return Long.valueOf(s);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-      if (TIMESTAMP_PATTERN.matcher(s).matches()) {
-        return DateTimeUtils.timestampStringToUnixDate(s);
-      }
-    }
-    return null;
-  }
-
-
-  private static Integer toInt(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).intValue();
-    }
-    if (literal instanceof String) {
-      try {
-        return Integer.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-    }
-    return null;
-  }
-
-  private static Float toFloat(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).floatValue();
-    }
-    if (literal instanceof String) {
-      try {
-        return Float.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
+  private static Calendar literalValue(RexNode node) {
+    if (node instanceof RexLiteral) {
+      return (Calendar) ((RexLiteral) node).getValue();
     }
     return null;
   }
 
-  private static Double toDouble(Object literal) {
-    if (literal instanceof Number) {
-      return ((Number) literal).doubleValue();
-    }
-    if (literal instanceof String) {
-      try {
-        return Double.valueOf((String) literal);
-      } catch (NumberFormatException e) {
-        // ignore
-      }
-    }
-    return null;
-  }
-
-  /**
-   * Extract the total time span covered by these intervals. It does not check
-   * if the intervals overlap.
-   * @param intervals list of intervals
-   * @return total time span covered by these intervals
-   */
-  public static long extractTotalTime(List<Interval> intervals) {
-    long totalTime = 0;
-    for (Interval interval : intervals) {
-      totalTime += interval.getEndMillis() - interval.getStartMillis();
-    }
-    return totalTime;
-  }
-
   /**
    * Extracts granularity from a call {@code FLOOR(<time> TO <timeunit>)}.
    * Timeunit specifies the granularity. Returns null if it cannot


[3/3] calcite git commit: [CALCITE-1454] Allow custom implementations of SqlConformance

Posted by jh...@apache.org.
[CALCITE-1454] Allow custom implementations of SqlConformance

SqlParser.Config.allowBangEqual() is now obsolete; use
SqlParser.Config.conformance().isBangEqualAllowed().


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

Branch: refs/heads/master
Commit: 105bba1f83cd9631e8e1211d262e4886a4a863b7
Parents: 54556b8
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Oct 17 21:30:30 2016 -0700
Committer: Julian Hyde <jh...@apache.org>
Committed: Tue Oct 18 10:25:55 2016 -0700

----------------------------------------------------------------------
 core/src/main/codegen/templates/Parser.jj       |  20 +---
 .../config/CalciteConnectionConfigImpl.java     |   3 +-
 .../config/CalciteConnectionProperty.java       |   4 +-
 .../calcite/jdbc/CalciteConnectionImpl.java     |   4 +-
 .../calcite/prepare/CalcitePrepareImpl.java     |   2 +-
 .../org/apache/calcite/prepare/PlannerImpl.java |   3 +-
 .../sql/parser/SqlAbstractParserImpl.java       |   5 +-
 .../apache/calcite/sql/parser/SqlParser.java    |  50 ++++++--
 .../sql/validate/SqlAbstractConformance.java    |  47 ++++++++
 .../calcite/sql/validate/SqlConformance.java    | 119 ++++++++++---------
 .../sql/validate/SqlConformanceEnum.java        | 102 ++++++++++++++++
 .../sql/validate/SqlDelegatingConformance.java  |  53 +++++++++
 .../calcite/sql/validate/SqlValidator.java      |   2 +-
 .../calcite/sql/validate/SqlValidatorUtil.java  |   2 +-
 .../calcite/sql/test/DefaultSqlTestFactory.java |   5 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |   6 +-
 .../apache/calcite/sql/test/SqlTesterImpl.java  |   3 +-
 .../apache/calcite/test/SqlToRelTestBase.java   |   3 +-
 .../apache/calcite/test/SqlValidatorTest.java   |  55 ++++++++-
 19 files changed, 388 insertions(+), 100 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/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 98466b2..6f964f3 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -96,10 +96,10 @@ import org.apache.calcite.sql.parser.SqlParserImplFactory;
 import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
-import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
 import org.slf4j.Logger;
@@ -135,7 +135,7 @@ public class ${parser.class} extends SqlAbstractParserImpl
     private Casing unquotedCasing;
     private Casing quotedCasing;
     private int identifierMaxLength;
-    private boolean allowBangEqual;
+    private SqlConformance conformance;
 
     /**
      * {@link SqlParserImplFactory} implementation for creating parser.
@@ -146,7 +146,6 @@ public class ${parser.class} extends SqlAbstractParserImpl
         }
     };
 
-    // implement SqlAbstractParserImpl
     public SqlParseException normalizeException(Throwable ex)
     {
         try {
@@ -159,7 +158,6 @@ public class ${parser.class} extends SqlAbstractParserImpl
         }
     }
 
-    // implement SqlAbstractParserImpl
     public Metadata getMetadata()
     {
         synchronized (${parser.class}.class) {
@@ -171,13 +169,11 @@ public class ${parser.class} extends SqlAbstractParserImpl
         }
     }
 
-    // implement SqlAbstractParserImpl
     public void setTabSize(int tabSize)
     {
         jj_input_stream.setTabSize(tabSize);
     }
 
-    // implement SqlAbstractParserImpl
     public void switchTo(String stateName)
     {
         int state = Arrays.asList(${parser.class}TokenManager.lexStateNames)
@@ -185,37 +181,31 @@ public class ${parser.class} extends SqlAbstractParserImpl
         token_source.SwitchTo(state);
     }
 
-    // implement SqlAbstractParserImpl
     public void setQuotedCasing(Casing quotedCasing)
     {
         this.quotedCasing = quotedCasing;
     }
 
-    // implement SqlAbstractParserImpl
     public void setUnquotedCasing(Casing unquotedCasing)
     {
         this.unquotedCasing = unquotedCasing;
     }
 
-    // implement SqlAbstractParserImpl
     public void setIdentifierMaxLength(int identifierMaxLength)
     {
         this.identifierMaxLength = identifierMaxLength;
     }
 
-    // implement SqlAbstractParserImpl
-    public void setAllowBangEqual(boolean allowBangEqual)
+    public void setConformance(SqlConformance conformance)
     {
-        this.allowBangEqual = allowBangEqual;
+        this.conformance = conformance;
     }
 
-    // implement SqlAbstractParserImpl
     public SqlNode parseSqlExpressionEof() throws Exception
     {
         return SqlExpressionEof();
     }
 
-    // implement SqlAbstractParserImpl
     public SqlNode parseSqlStmtEof() throws Exception
     {
         return SqlStmtEof();
@@ -4742,7 +4732,7 @@ SqlBinaryOperator BinaryRowOperator() :
     { return SqlStdOperatorTable.NOT_EQUALS; }
     | <NE2>
     {
-        if (!this.allowBangEqual) {
+        if (!this.conformance.isBangEqualAllowed()) {
             throw new ParseException(RESOURCE.bangEqualNotAllowed().str());
         }
         return SqlStdOperatorTable.NOT_EQUALS;

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
index 9af61f5..f3dbd2a 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionConfigImpl.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.fun.OracleSqlOperatorTable;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -150,7 +151,7 @@ public class CalciteConnectionConfigImpl extends ConnectionConfigImpl
 
   public SqlConformance conformance() {
     return CalciteConnectionProperty.CONFORMANCE.wrap(properties)
-        .getEnum(SqlConformance.class);
+        .getEnum(SqlConformanceEnum.class);
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
index db88abc..2086f87 100644
--- a/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
+++ b/core/src/main/java/org/apache/calcite/config/CalciteConnectionProperty.java
@@ -18,7 +18,7 @@ package org.apache.calcite.config;
 
 import org.apache.calcite.avatica.ConnectionProperty;
 import org.apache.calcite.model.JsonSchema;
-import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 
 import java.util.HashMap;
 import java.util.Map;
@@ -114,7 +114,7 @@ public enum CalciteConnectionProperty implements ConnectionProperty {
   TYPE_SYSTEM("typeSystem", Type.PLUGIN, null, false),
 
   /** SQL conformance level. */
-  CONFORMANCE("conformance", Type.ENUM, SqlConformance.DEFAULT, false);
+  CONFORMANCE("conformance", Type.ENUM, SqlConformanceEnum.DEFAULT, false);
 
   private final String camelName;
   private final Type type;

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
index 195ced7..5eebb38 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -54,7 +54,7 @@ import org.apache.calcite.server.CalciteServerStatement;
 import org.apache.calcite.sql.advise.SqlAdvisor;
 import org.apache.calcite.sql.advise.SqlAdvisorValidator;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
-import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidatorWithHints;
 import org.apache.calcite.tools.RelRunner;
 import org.apache.calcite.util.BuiltInMethod;
@@ -430,7 +430,7 @@ abstract class CalciteConnectionImpl
           new SqlAdvisorValidator(SqlStdOperatorTable.instance(),
           new CalciteCatalogReader(rootSchema, con.config().caseSensitive(),
               schemaPath, typeFactory),
-          typeFactory, SqlConformance.DEFAULT);
+          typeFactory, SqlConformanceEnum.DEFAULT);
       return new SqlAdvisor(validator);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
index 360677c..fcbee23 100644
--- a/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/CalcitePrepareImpl.java
@@ -699,7 +699,7 @@ public class CalcitePrepareImpl implements CalcitePrepare {
               .setQuotedCasing(config.quotedCasing())
               .setUnquotedCasing(config.unquotedCasing())
               .setQuoting(config.quoting())
-              .setAllowBangEqual(config.conformance().isBangEqualAllowed()));
+              .setConformance(config.conformance()));
       SqlNode sqlNode;
       try {
         sqlNode = parser.parseStmt();

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
index 3291ceb..635b2c5 100644
--- a/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
+++ b/core/src/main/java/org/apache/calcite/prepare/PlannerImpl.java
@@ -37,6 +37,7 @@ import org.apache.calcite.sql.SqlOperatorTable;
 import org.apache.calcite.sql.parser.SqlParseException;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql2rel.RelDecorrelator;
 import org.apache.calcite.sql2rel.SqlRexConvertletTable;
@@ -198,7 +199,7 @@ public class PlannerImpl implements Planner {
         return connectionConfig.conformance();
       }
     }
-    return SqlConformance.DEFAULT;
+    return SqlConformanceEnum.DEFAULT;
   }
 
   public Pair<SqlNode, RelDataType> validateAndGetType(SqlNode sqlNode)

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
index c7cef03..ad605da 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlAbstractParserImpl.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.sql.SqlUnresolvedFunction;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.sql.validate.SqlConformance;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -444,9 +445,9 @@ public abstract class SqlAbstractParserImpl {
   public abstract void setIdentifierMaxLength(int identifierMaxLength);
 
   /**
-   * Sets whether the bang-equal token != is allowed as an alternative to &lt;&gt;.
+   * Sets the SQL language conformance level.
    */
-  public abstract void setAllowBangEqual(boolean allowBangEqual);
+  public abstract void setConformance(SqlConformance conformance);
 
   /**
    * Sets the SQL text that is being parsed.

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
index a1311a0..626269a 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParser.java
@@ -22,6 +22,9 @@ import org.apache.calcite.config.Lex;
 import org.apache.calcite.runtime.CalciteContextException;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.parser.impl.SqlParserImpl;
+import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.calcite.sql.validate.SqlDelegatingConformance;
 
 import com.google.common.base.Preconditions;
 
@@ -32,7 +35,9 @@ import java.io.StringReader;
  */
 public class SqlParser {
   public static final int DEFAULT_IDENTIFIER_MAX_LENGTH = 128;
-  public static final boolean DEFAULT_ALLOW_BANG_EQUAL = false;
+  @Deprecated // to be removed before 2.0
+  public static final boolean DEFAULT_ALLOW_BANG_EQUAL =
+      SqlConformanceEnum.DEFAULT.isBangEqualAllowed();
 
   //~ Instance fields --------------------------------------------------------
   private final SqlAbstractParserImpl parser;
@@ -48,7 +53,7 @@ public class SqlParser {
     parser.setQuotedCasing(config.quotedCasing());
     parser.setUnquotedCasing(config.unquotedCasing());
     parser.setIdentifierMaxLength(config.identifierMaxLength());
-    parser.setAllowBangEqual(config.allowBangEqual());
+    parser.setConformance(config.conformance());
     switch (config.quoting()) {
     case DOUBLE_QUOTE:
       parser.switchTo("DQID");
@@ -196,6 +201,8 @@ public class SqlParser {
     Casing unquotedCasing();
     Quoting quoting();
     boolean caseSensitive();
+    SqlConformance conformance();
+    @Deprecated // to be removed before 2.0
     boolean allowBangEqual();
     SqlParserImplFactory parserFactory();
   }
@@ -207,7 +214,9 @@ public class SqlParser {
     private Quoting quoting = Lex.ORACLE.quoting;
     private int identifierMaxLength = DEFAULT_IDENTIFIER_MAX_LENGTH;
     private boolean caseSensitive = Lex.ORACLE.caseSensitive;
-    private boolean allowBangEqual = DEFAULT_ALLOW_BANG_EQUAL;
+    private SqlConformance conformance = SqlConformanceEnum.DEFAULT;
+    private boolean allowBangEqual =
+        SqlConformanceEnum.DEFAULT.isBangEqualAllowed();
     private SqlParserImplFactory parserFactory = SqlParserImpl.FACTORY;
 
     private ConfigBuilder() {}
@@ -218,6 +227,7 @@ public class SqlParser {
       this.unquotedCasing = config.unquotedCasing();
       this.quoting = config.quoting();
       this.identifierMaxLength = config.identifierMaxLength();
+      this.conformance = config.conformance();
       this.allowBangEqual = config.allowBangEqual();
       this.parserFactory = config.parserFactory();
       return this;
@@ -248,8 +258,23 @@ public class SqlParser {
       return this;
     }
 
-    public ConfigBuilder setAllowBangEqual(boolean allowBangEqual) {
-      this.allowBangEqual = allowBangEqual;
+    @SuppressWarnings("unused")
+    @Deprecated // to be removed before 2.0
+    public ConfigBuilder setAllowBangEqual(final boolean allowBangEqual) {
+      if (allowBangEqual != conformance.isBangEqualAllowed()) {
+        setConformance(
+            new SqlDelegatingConformance(conformance) {
+              @Override public boolean isBangEqualAllowed() {
+                return allowBangEqual;
+              }
+            });
+      }
+      return this;
+    }
+
+    public ConfigBuilder setConformance(SqlConformance conformance) {
+      this.conformance = conformance;
+      this.allowBangEqual = conformance.isBangEqualAllowed();
       return this;
     }
 
@@ -270,8 +295,9 @@ public class SqlParser {
      * {@link Config}. */
     public Config build() {
       return new ConfigImpl(identifierMaxLength, quotedCasing, unquotedCasing,
-          quoting, caseSensitive, allowBangEqual, parserFactory);
+          quoting, caseSensitive, conformance, parserFactory);
     }
+
   }
 
   /** Implementation of
@@ -280,7 +306,7 @@ public class SqlParser {
   private static class ConfigImpl implements Config {
     private final int identifierMaxLength;
     private final boolean caseSensitive;
-    private final boolean allowBangEqual;
+    private final SqlConformance conformance;
     private final Casing quotedCasing;
     private final Casing unquotedCasing;
     private final Quoting quoting;
@@ -288,10 +314,10 @@ public class SqlParser {
 
     private ConfigImpl(int identifierMaxLength, Casing quotedCasing,
         Casing unquotedCasing, Quoting quoting, boolean caseSensitive,
-        boolean allowBangEqual, SqlParserImplFactory parserFactory) {
+        SqlConformance conformance, SqlParserImplFactory parserFactory) {
       this.identifierMaxLength = identifierMaxLength;
       this.caseSensitive = caseSensitive;
-      this.allowBangEqual = allowBangEqual;
+      this.conformance = Preconditions.checkNotNull(conformance);
       this.quotedCasing = Preconditions.checkNotNull(quotedCasing);
       this.unquotedCasing = Preconditions.checkNotNull(unquotedCasing);
       this.quoting = Preconditions.checkNotNull(quoting);
@@ -318,8 +344,12 @@ public class SqlParser {
       return caseSensitive;
     }
 
+    public SqlConformance conformance() {
+      return conformance;
+    }
+
     public boolean allowBangEqual() {
-      return allowBangEqual;
+      return conformance.isBangEqualAllowed();
     }
 
     public SqlParserImplFactory parserFactory() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
new file mode 100644
index 0000000..a36770c
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlAbstractConformance.java
@@ -0,0 +1,47 @@
+/*
+ * 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.validate;
+
+/**
+ * Abstract base class for implementing {@link SqlConformance}.
+ *
+ * <p>Every method in {@code SqlConformance} is implemented,
+ * and behaves the same as in {@link SqlConformanceEnum#DEFAULT}.
+ */
+public abstract class SqlAbstractConformance implements SqlConformance {
+  public boolean isSortByOrdinal() {
+    return SqlConformanceEnum.DEFAULT.isSortByOrdinal();
+  }
+
+  public boolean isSortByAlias() {
+    return SqlConformanceEnum.DEFAULT.isSortByAlias();
+  }
+
+  public boolean isSortByAliasObscures() {
+    return SqlConformanceEnum.DEFAULT.isSortByAliasObscures();
+  }
+
+  public boolean isFromRequired() {
+    return SqlConformanceEnum.DEFAULT.isFromRequired();
+  }
+
+  public boolean isBangEqualAllowed() {
+    return SqlConformanceEnum.DEFAULT.isBangEqualAllowed();
+  }
+}
+
+// End SqlAbstractConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
index 2b72499..22ac06c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformance.java
@@ -18,95 +18,106 @@ package org.apache.calcite.sql.validate;
 
 /**
  * Enumeration of valid SQL compatibility modes.
+ *
+ * <p>For most purposes, one of the built-in compatibility modes in enum
+ * {@link SqlConformanceEnum} will suffice.
+ *
+ * <p>If you wish to implement this interface to build your own conformance,
+ * we strongly recommend that you extend {@link SqlAbstractConformance},
+ * or use a {@link SqlDelegatingConformance},
+ * so that you won't be broken by future changes.
+ *
+ * @see SqlConformanceEnum
+ * @see SqlAbstractConformance
+ * @see SqlDelegatingConformance
  */
-public enum SqlConformance {
-  DEFAULT, STRICT_92, STRICT_99, PRAGMATIC_99, ORACLE_10, STRICT_2003,
-  PRAGMATIC_2003;
+public interface SqlConformance {
+  /** Short-cut for {@link SqlConformanceEnum#DEFAULT}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum DEFAULT = SqlConformanceEnum.DEFAULT;
+  /** Short-cut for {@link SqlConformanceEnum#STRICT_92}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum STRICT_92 = SqlConformanceEnum.STRICT_92;
+  /** Short-cut for {@link SqlConformanceEnum#STRICT_99}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum STRICT_99 = SqlConformanceEnum.STRICT_99;
+  /** Short-cut for {@link SqlConformanceEnum#PRAGMATIC_99}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum PRAGMATIC_99 = SqlConformanceEnum.PRAGMATIC_99;
+  /** Short-cut for {@link SqlConformanceEnum#ORACLE_10}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum ORACLE_10 = SqlConformanceEnum.ORACLE_10;
+  /** Short-cut for {@link SqlConformanceEnum#STRICT_2003}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum STRICT_2003 = SqlConformanceEnum.STRICT_2003;
+  /** Short-cut for {@link SqlConformanceEnum#PRAGMATIC_2003}. */
+  @SuppressWarnings("unused")
+  @Deprecated // to be removed before 2.0
+  SqlConformanceEnum PRAGMATIC_2003 = SqlConformanceEnum.PRAGMATIC_2003;
 
   /**
    * Whether 'order by 2' is interpreted to mean 'sort by the 2nd column in
    * the select list'.
    *
-   * <p>True in {@link #DEFAULT}, {@link #ORACLE_10}, {@link #STRICT_92},
-   * {@link #PRAGMATIC_99}, {@link #PRAGMATIC_2003};
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#DEFAULT},
+   * {@link SqlConformanceEnum#ORACLE_10},
+   * {@link SqlConformanceEnum#STRICT_92},
+   * {@link SqlConformanceEnum#PRAGMATIC_99},
+   * {@link SqlConformanceEnum#PRAGMATIC_2003};
    * false otherwise.
    */
-  public boolean isSortByOrdinal() {
-    switch (this) {
-    case DEFAULT:
-    case ORACLE_10:
-    case STRICT_92:
-    case PRAGMATIC_99:
-    case PRAGMATIC_2003:
-      return true;
-    default:
-      return false;
-    }
-  }
+  boolean isSortByOrdinal();
 
   /**
    * Whether 'order by x' is interpreted to mean 'sort by the select list item
    * whose alias is x' even if there is a column called x.
    *
-   * <p>True in {@link #DEFAULT}, {@link #ORACLE_10}, {@link #STRICT_92};
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#DEFAULT},
+   * {@link SqlConformanceEnum#ORACLE_10},
+   * {@link SqlConformanceEnum#STRICT_92};
    * false otherwise.
    */
-  public boolean isSortByAlias() {
-    switch (this) {
-    case DEFAULT:
-    case ORACLE_10:
-    case STRICT_92:
-      return true;
-    default:
-      return false;
-    }
-  }
+  boolean isSortByAlias();
 
   /**
    * Whether "empno" is invalid in "select empno as x from emp order by empno"
    * because the alias "x" obscures it.
    *
-   * <p>True in {@link #STRICT_92};
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#STRICT_92};
    * false otherwise.
    */
-  public boolean isSortByAliasObscures() {
-    return this == SqlConformance.STRICT_92;
-  }
+  boolean isSortByAliasObscures();
 
   /**
    * Whether FROM clause is required in a SELECT statement.
    *
-   * <p>True in {@link #ORACLE_10}, {@link #STRICT_92}, {@link #STRICT_99},
-   * {@link #STRICT_2003};
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#ORACLE_10},
+   * {@link SqlConformanceEnum#STRICT_92},
+   * {@link SqlConformanceEnum#STRICT_99},
+   * {@link SqlConformanceEnum#STRICT_2003};
    * false otherwise.
    */
-  public boolean isFromRequired() {
-    switch (this) {
-    case ORACLE_10:
-    case STRICT_92:
-    case STRICT_99:
-    case STRICT_2003:
-      return true;
-    default:
-      return false;
-    }
-  }
+  boolean isFromRequired();
 
   /**
    * Whether the bang-equal token != is allowed as an alternative to &lt;&gt; in
    * the parser.
    *
-   * <p>True in {@link #ORACLE_10};
+   * <p>Among the built-in conformance levels, true in
+   * {@link SqlConformanceEnum#ORACLE_10};
    * false otherwise.
    */
-  public boolean isBangEqualAllowed() {
-    switch (this) {
-    case ORACLE_10:
-      return true;
-    default:
-      return false;
-    }
-  }
+  boolean isBangEqualAllowed();
 }
 
 // End SqlConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
new file mode 100644
index 0000000..5458893
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlConformanceEnum.java
@@ -0,0 +1,102 @@
+/*
+ * 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.validate;
+
+/**
+ * Enumeration of built-in SQL compatibility modes.
+ */
+public enum SqlConformanceEnum implements SqlConformance {
+  /** Calcite's default SQL behavior. */
+  DEFAULT,
+
+  /** Conformance value that instructs Calcite to use SQL semantics strictly
+   * consistent with the SQL:92 standard. */
+  STRICT_92,
+
+  /** Conformance value that instructs Calcite to use SQL semantics strictly
+   * consistent with the SQL:99 standard. */
+  STRICT_99,
+
+  /** Conformance value that instructs Calcite to use SQL semantics
+   * consistent with the SQL:99 standard, but ignoring its more
+   * inconvenient or controversial dicta. */
+  PRAGMATIC_99,
+
+  /** Conformance value that instructs Calcite to use SQL semantics
+   * consistent with Oracle version 10. */
+  ORACLE_10,
+
+  /** Conformance value that instructs Calcite to use SQL semantics strictly
+   * consistent with the SQL:2003 standard. */
+  STRICT_2003,
+
+  /** Conformance value that instructs Calcite to use SQL semantics
+   * consistent with the SQL:2003 standard, but ignoring its more
+   * inconvenient or controversial dicta. */
+  PRAGMATIC_2003;
+
+  public boolean isSortByOrdinal() {
+    switch (this) {
+    case DEFAULT:
+    case ORACLE_10:
+    case STRICT_92:
+    case PRAGMATIC_99:
+    case PRAGMATIC_2003:
+      return true;
+    default:
+      return false;
+    }
+  }
+
+  public boolean isSortByAlias() {
+    switch (this) {
+    case DEFAULT:
+    case ORACLE_10:
+    case STRICT_92:
+      return true;
+    default:
+      return false;
+    }
+  }
+
+  public boolean isSortByAliasObscures() {
+    return this == SqlConformanceEnum.STRICT_92;
+  }
+
+  public boolean isFromRequired() {
+    switch (this) {
+    case ORACLE_10:
+    case STRICT_92:
+    case STRICT_99:
+    case STRICT_2003:
+      return true;
+    default:
+      return false;
+    }
+  }
+
+  public boolean isBangEqualAllowed() {
+    switch (this) {
+    case ORACLE_10:
+      return true;
+    default:
+      return false;
+    }
+  }
+}
+
+// End SqlConformanceEnum.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/validate/SqlDelegatingConformance.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlDelegatingConformance.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlDelegatingConformance.java
new file mode 100644
index 0000000..47d56d3
--- /dev/null
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlDelegatingConformance.java
@@ -0,0 +1,53 @@
+/*
+ * 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.validate;
+
+/**
+ * Implementation of {@link SqlConformance} that delegates all methods to
+ * another object. You can create a sub-class that overrides particular
+ * methods.
+ */
+public class SqlDelegatingConformance extends SqlAbstractConformance {
+  private final SqlConformance delegate;
+
+  /** Creates a SqlDelegatingConformance. */
+  protected SqlDelegatingConformance(SqlConformance delegate) {
+    this.delegate = delegate;
+  }
+
+  @Override public boolean isSortByOrdinal() {
+    return delegate.isSortByOrdinal();
+  }
+
+  @Override public boolean isSortByAlias() {
+    return delegate.isSortByAlias();
+  }
+
+  @Override public boolean isSortByAliasObscures() {
+    return delegate.isSortByAliasObscures();
+  }
+
+  @Override public boolean isFromRequired() {
+    return delegate.isFromRequired();
+  }
+
+  @Override public boolean isBangEqualAllowed() {
+    return delegate.isBangEqualAllowed();
+  }
+}
+
+// End SqlDelegatingConformance.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
index 605c665..215b96c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidator.java
@@ -109,7 +109,7 @@ public interface SqlValidator {
 
   /**
    * Returns the dialect of SQL (SQL:2003, etc.) this validator recognizes.
-   * Default is {@link SqlConformance#DEFAULT}.
+   * Default is {@link SqlConformanceEnum#DEFAULT}.
    *
    * @return dialect of SQL this validator recognizes
    */

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
index 249f5f5..62410b3 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorUtil.java
@@ -209,7 +209,7 @@ public class SqlValidatorUtil {
       SqlValidatorCatalogReader catalogReader,
       RelDataTypeFactory typeFactory) {
     return newValidator(opTab, catalogReader, typeFactory,
-        SqlConformance.DEFAULT);
+        SqlConformanceEnum.DEFAULT);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
index c9ecce6..be651ff 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/DefaultSqlTestFactory.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.sql.validate.SqlValidatorWithHints;
@@ -51,7 +52,7 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
           .put("quotedCasing", Casing.UNCHANGED)
           .put("unquotedCasing", Casing.TO_UPPER)
           .put("caseSensitive", true)
-          .put("conformance", SqlConformance.DEFAULT)
+          .put("conformance", SqlConformanceEnum.DEFAULT)
           .put("operatorTable", SqlStdOperatorTable.instance())
           .put("connectionFactory",
               CalciteAssert.EMPTY_CONNECTION_FACTORY
@@ -80,7 +81,7 @@ public class DefaultSqlTestFactory implements SqlTestFactory {
             .setQuoting((Quoting) factory.get("quoting"))
             .setUnquotedCasing((Casing) factory.get("unquotedCasing"))
             .setQuotedCasing((Casing) factory.get("quotedCasing"))
-            .setAllowBangEqual(((SqlConformance) factory.get("conformance")).isBangEqualAllowed())
+            .setConformance((SqlConformance) factory.get("conformance"))
             .build());
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/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 0c3acc6..3c024f8 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
@@ -42,7 +42,7 @@ import org.apache.calcite.sql.type.SqlOperandTypeChecker;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
 import org.apache.calcite.sql.util.SqlString;
-import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.test.CalciteAssert;
@@ -2775,10 +2775,10 @@ public abstract class SqlOperatorBaseTest {
     // "!=" is allowed under ORACLE_10 SQL conformance level
     final SqlTester tester1 =
         tester
-            .withConformance(SqlConformance.ORACLE_10)
+            .withConformance(SqlConformanceEnum.ORACLE_10)
             .withConnectionFactory(
                 CalciteAssert.EMPTY_CONNECTION_FACTORY
-                    .with("conformance", SqlConformance.ORACLE_10));
+                    .with("conformance", SqlConformanceEnum.ORACLE_10));
 
     tester1
         .checkBoolean("1 <> 1", Boolean.FALSE);

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
index 3c55d6d..fdf08f8 100644
--- a/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
+++ b/core/src/test/java/org/apache/calcite/sql/test/SqlTesterImpl.java
@@ -39,6 +39,7 @@ import org.apache.calcite.sql.parser.SqlParserUtil;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlShuttle;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorNamespace;
@@ -290,7 +291,7 @@ public class SqlTesterImpl implements SqlTester {
 
   public SqlTesterImpl withConformance(SqlConformance conformance) {
     if (conformance == null) {
-      conformance = SqlConformance.DEFAULT;
+      conformance = SqlConformanceEnum.DEFAULT;
     }
     return with("conformance", conformance);
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
index d8e0ff0..166db72 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlToRelTestBase.java
@@ -44,6 +44,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.parser.SqlParser;
 import org.apache.calcite.sql.type.SqlTypeFactoryImpl;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorCatalogReader;
@@ -591,7 +592,7 @@ public abstract class SqlToRelTestBase {
     }
 
     public SqlConformance getConformance() {
-      return SqlConformance.DEFAULT;
+      return SqlConformanceEnum.DEFAULT;
     }
 
     public SqlValidator createValidator(

http://git-wip-us.apache.org/repos/asf/calcite/blob/105bba1f/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 a6d0c35..caee32f 100644
--- a/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
+++ b/core/src/test/java/org/apache/calcite/test/SqlValidatorTest.java
@@ -29,7 +29,10 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.test.SqlTester;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.ChainedSqlOperatorTable;
+import org.apache.calcite.sql.validate.SqlAbstractConformance;
 import org.apache.calcite.sql.validate.SqlConformance;
+import org.apache.calcite.sql.validate.SqlConformanceEnum;
+import org.apache.calcite.sql.validate.SqlDelegatingConformance;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
@@ -46,11 +49,14 @@ import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 import java.nio.charset.Charset;
 import java.util.Arrays;
 import java.util.Comparator;
 import java.util.List;
 import java.util.Locale;
+import java.util.Objects;
 
 import static org.hamcrest.CoreMatchers.equalTo;
 import static org.hamcrest.CoreMatchers.is;
@@ -5731,6 +5737,49 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
     return buf.toString();
   }
 
+  @Test public void testAbstractConformance()
+      throws InvocationTargetException, IllegalAccessException {
+    final SqlAbstractConformance c0 = new SqlAbstractConformance() {
+    };
+    final SqlConformance c1 = SqlConformanceEnum.DEFAULT;
+    for (Method method : SqlConformance.class.getMethods()) {
+      final Object o0 = method.invoke(c0);
+      final Object o1 = method.invoke(c1);
+      assertThat(method.toString(), Objects.equals(o0, o1), is(true));
+    }
+  }
+
+  @Test public void testUserDefinedConformance() {
+    final SqlAbstractConformance c =
+        new SqlDelegatingConformance(SqlConformanceEnum.DEFAULT) {
+          public boolean isBangEqualAllowed() {
+            return true;
+          }
+        };
+    // Our conformance behaves differently from ORACLE_10 for FROM-less query.
+    final SqlTester customTester = tester.withConformance(c);
+    final SqlTester defaultTester =
+        tester.withConformance(SqlConformanceEnum.DEFAULT);
+    final SqlTester oracleTester =
+        tester.withConformance(SqlConformanceEnum.ORACLE_10);
+    sql("^select 2+2^")
+        .tester(customTester)
+        .ok()
+        .tester(defaultTester)
+        .ok()
+        .tester(oracleTester)
+        .fails("SELECT must have a FROM clause");
+
+    // Our conformance behaves like ORACLE_10 for "!=" operator.
+    sql("select * from (values 1) where 1 != 2")
+        .tester(customTester)
+        .ok()
+        .tester(defaultTester)
+        .fails("Bang equal '!=' is not allowed under the current SQL conformance level")
+        .tester(oracleTester)
+        .ok();
+  }
+
   @Test public void testOrder() {
     final SqlConformance conformance = tester.getConformance();
     check("select empno as x from emp order by empno");
@@ -7016,13 +7065,13 @@ public class SqlValidatorTest extends SqlValidatorTestCase {
 
   @Test public void testSelectWithoutFrom() {
     sql("^select 2+2^")
-        .tester(tester.withConformance(SqlConformance.DEFAULT))
+        .tester(tester.withConformance(SqlConformanceEnum.DEFAULT))
         .ok();
     sql("^select 2+2^")
-        .tester(tester.withConformance(SqlConformance.ORACLE_10))
+        .tester(tester.withConformance(SqlConformanceEnum.ORACLE_10))
         .fails("SELECT must have a FROM clause");
     sql("^select 2+2^")
-        .tester(tester.withConformance(SqlConformance.STRICT_2003))
+        .tester(tester.withConformance(SqlConformanceEnum.STRICT_2003))
         .fails("SELECT must have a FROM clause");
   }