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/01/21 23:38:53 UTC

[16/50] [abbrv] calcite git commit: [CALCITE-955] Litmus (continuation-passing style for methods that check invariants)

[CALCITE-955] Litmus (continuation-passing style for methods that check invariants)


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

Branch: refs/heads/branch-release
Commit: 9a52b5eda9fcfb4e6a7ad94fbd4a1c3fa2fb869a
Parents: b5b28f0
Author: Julian Hyde <jh...@apache.org>
Authored: Thu Nov 5 18:09:38 2015 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Sun Jan 10 00:51:24 2016 -0800

----------------------------------------------------------------------
 .../org/apache/calcite/plan/RelOptUtil.java     |  40 +++----
 .../calcite/plan/SubstitutionVisitor.java       |  19 ++--
 .../apache/calcite/plan/volcano/RelSubset.java  |   7 +-
 .../calcite/plan/volcano/VolcanoPlanner.java    |  38 +++----
 .../org/apache/calcite/rel/AbstractRelNode.java |  11 +-
 .../java/org/apache/calcite/rel/RelNode.java    |  16 ++-
 .../org/apache/calcite/rel/core/Aggregate.java  |  22 ++--
 .../java/org/apache/calcite/rel/core/Calc.java  |  20 ++--
 .../org/apache/calcite/rel/core/Filter.java     |  15 ++-
 .../java/org/apache/calcite/rel/core/Join.java  |  23 ++--
 .../org/apache/calcite/rel/core/Project.java    | 114 +++----------------
 .../org/apache/calcite/rel/core/Window.java     |  12 +-
 .../calcite/rel/logical/LogicalWindow.java      |   5 +-
 .../calcite/rel/rules/CalcRelSplitter.java      |   3 +-
 .../java/org/apache/calcite/rex/RexCall.java    |   3 +-
 .../java/org/apache/calcite/rex/RexChecker.java |  48 ++++----
 .../java/org/apache/calcite/rex/RexProgram.java | 113 +++++++++---------
 .../apache/calcite/rex/RexProgramBuilder.java   |  15 +--
 .../java/org/apache/calcite/rex/RexUtil.java    |  77 ++++++-------
 .../org/apache/calcite/schema/SchemaPlus.java   |  11 +-
 .../apache/calcite/sql/SqlBinaryOperator.java   |   8 +-
 .../java/org/apache/calcite/sql/SqlCall.java    |  11 +-
 .../org/apache/calcite/sql/SqlDataTypeSpec.java |  29 ++---
 .../org/apache/calcite/sql/SqlDynamicParam.java |  11 +-
 .../org/apache/calcite/sql/SqlIdentifier.java   |  14 +--
 .../calcite/sql/SqlIntervalQualifier.java       |   8 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java |  11 +-
 .../java/org/apache/calcite/sql/SqlNode.java    |  29 +++--
 .../org/apache/calcite/sql/SqlNodeList.java     |  19 ++--
 .../org/apache/calcite/sql/SqlOperator.java     |   3 +-
 .../apache/calcite/sql/SqlPostfixOperator.java  |   8 +-
 .../apache/calcite/sql/SqlPrefixOperator.java   |   8 +-
 .../java/org/apache/calcite/sql/SqlWindow.java  |   9 +-
 .../apache/calcite/sql/fun/SqlInOperator.java   |  10 +-
 .../calcite/sql/fun/SqlStdOperatorTable.java    |   8 ++
 .../calcite/sql2rel/SqlToRelConverter.java      |   8 +-
 .../java/org/apache/calcite/util/Litmus.java    |  36 +++++-
 .../main/java/org/apache/calcite/util/Util.java |   2 +
 .../calcite/test/SqlToRelConverterTest.java     |   5 +-
 39 files changed, 404 insertions(+), 445 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
index 46a1dbc..b932cef 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -72,6 +72,7 @@ import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 import org.apache.calcite.util.Util;
@@ -1625,7 +1626,7 @@ public abstract class RelOptUtil {
    * @param type1 First type
    * @param desc2 Description of second type
    * @param type2 Second type
-   * @param fail  Whether to assert if they are not equal
+   * @param litmus What to do if an error is detected (types are not equal)
    * @return Whether the types are equal
    */
   public static boolean eq(
@@ -1633,22 +1634,21 @@ public abstract class RelOptUtil {
       RelDataType type1,
       final String desc2,
       RelDataType type2,
-      boolean fail) {
+      Litmus litmus) {
     // if any one of the types is ANY return true
     if (type1.getSqlTypeName() == SqlTypeName.ANY
         || type2.getSqlTypeName() == SqlTypeName.ANY) {
-      return true;
+      return litmus.succeed();
     }
 
     if (type1 != type2) {
-      assert !fail : "type mismatch:\n"
+      return litmus.fail("type mismatch:\n"
           + desc1 + ":\n"
           + type1.getFullTypeString() + "\n"
           + desc2 + ":\n"
-          + type2.getFullTypeString();
-      return false;
+          + type2.getFullTypeString());
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**
@@ -1660,7 +1660,7 @@ public abstract class RelOptUtil {
    * @param type1 First type
    * @param desc2 Description of role of second type
    * @param type2 Second type
-   * @param fail  Whether to assert if they are not equal
+   * @param litmus Whether to assert if they are not equal
    * @return Whether the types are equal
    */
   public static boolean equal(
@@ -1668,26 +1668,22 @@ public abstract class RelOptUtil {
       RelDataType type1,
       final String desc2,
       RelDataType type2,
-      boolean fail) {
+      Litmus litmus) {
     if (!areRowTypesEqual(type1, type2, false)) {
-      if (fail) {
-        throw new AssertionError(
-            "Type mismatch:\n"
-            + desc1 + ":\n"
-            + type1.getFullTypeString() + "\n"
-            + desc2 + ":\n"
-            + type2.getFullTypeString());
-      }
-      return false;
+      return litmus.fail("Type mismatch:\n"
+          + desc1 + ":\n"
+          + type1.getFullTypeString() + "\n"
+          + desc2 + ":\n"
+          + type2.getFullTypeString());
     }
-    return true;
+    return litmus.succeed();
   }
 
   /** Returns whether two relational expressions have the same row-type. */
   public static boolean equalType(String desc0, RelNode rel0, String desc1,
-      RelNode rel1, boolean fail) {
+      RelNode rel1, Litmus litmus) {
     // TODO: change 'equal' to 'eq', which is stronger.
-    return equal(desc0, rel0.getRowType(), desc1, rel1.getRowType(), fail);
+    return equal(desc0, rel0.getRowType(), desc1, rel1.getRowType(), litmus);
   }
 
   /**
@@ -2515,7 +2511,7 @@ public abstract class RelOptUtil {
       // Short-cut common case.
       return query;
     }
-    assert equalType("find", find, "replace", replace, true);
+    assert equalType("find", find, "replace", replace, Litmus.THROW);
     if (query == find) {
       // Short-cut another common case.
       return replace;

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
index c88f51d..8dd0d01 100644
--- a/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
+++ b/core/src/main/java/org/apache/calcite/plan/SubstitutionVisitor.java
@@ -54,6 +54,7 @@ import org.apache.calcite.util.Bug;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mapping;
@@ -405,7 +406,7 @@ public class SubstitutionVisitor {
     assert false; // not called
     MutableRel replacement = toMutable(replacement_);
     assert MutableRels.equalType(
-        "target", target, "replacement", replacement, true);
+        "target", target, "replacement", replacement, Litmus.THROW);
     replacementMap.put(target, replacement);
     final UnifyResult unifyResult = matchRecurse(target);
     if (unifyResult == null) {
@@ -458,7 +459,7 @@ public class SubstitutionVisitor {
    */
   private List<List<Replacement>> go(MutableRel replacement) {
     assert MutableRels.equalType(
-        "target", target, "replacement", replacement, true);
+        "target", target, "replacement", replacement, Litmus.THROW);
     final List<MutableRel> queryDescendants = MutableRels.descendants(query);
     final List<MutableRel> targetDescendants = MutableRels.descendants(target);
 
@@ -895,7 +896,8 @@ public class SubstitutionVisitor {
 
     public UnifyResult result(MutableRel result) {
       assert MutableRels.contains(result, target);
-      assert MutableRels.equalType("result", result, "query", query, true);
+      assert MutableRels.equalType("result", result, "query", query,
+          Litmus.THROW);
       MutableRel replace = replacementMap.get(target);
       if (replace != null) {
         assert false; // replacementMap is always empty
@@ -931,7 +933,8 @@ public class SubstitutionVisitor {
 
     UnifyResult(UnifyRuleCall call, MutableRel result) {
       this.call = call;
-      assert MutableRels.equalType("query", call.query, "result", result, true);
+      assert MutableRels.equalType("query", call.query, "result", result,
+          Litmus.THROW);
       this.result = result;
     }
   }
@@ -1661,7 +1664,7 @@ public class SubstitutionVisitor {
         List<RexNode> projects) {
       super(MutableRelType.PROJECT, rowType, input);
       this.projects = projects;
-      assert RexUtil.compatibleTypes(projects, rowType, true);
+      assert RexUtil.compatibleTypes(projects, rowType, Litmus.THROW);
     }
 
     public static MutableProject of(RelDataType rowType, MutableRel input,
@@ -2096,9 +2099,9 @@ public class SubstitutionVisitor {
 
     /** Returns whether two relational expressions have the same row-type. */
     public static boolean equalType(String desc0, MutableRel rel0, String desc1,
-        MutableRel rel1, boolean fail) {
+        MutableRel rel1, Litmus litmus) {
       return RelOptUtil.equal(desc0, rel0.getRowType(),
-          desc1, rel1.getRowType(), fail);
+          desc1, rel1.getRowType(), litmus);
     }
 
     /** Within a relational expression {@code query}, replaces occurrences of
@@ -2112,7 +2115,7 @@ public class SubstitutionVisitor {
         // Short-cut common case.
         return null;
       }
-      assert equalType("find", find, "replace", replace, true);
+      assert equalType("find", find, "replace", replace, Litmus.THROW);
       return replaceRecurse(query, find, replace);
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
index 0ecc264..bb42218 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/RelSubset.java
@@ -30,6 +30,7 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.RelWriter;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -271,10 +272,8 @@ public class RelSubset extends AbstractRelNode {
     // If this isn't the first rel in the set, it must have compatible
     // row type.
     if (set.rel != null) {
-      if (!RelOptUtil.equal("rowtype of new rel", rel.getRowType(),
-          "rowtype of set", getRowType(), true)) {
-        throw new AssertionError();
-      }
+      RelOptUtil.equal("rowtype of new rel", rel.getRowType(),
+          "rowtype of set", getRowType(), Litmus.THROW);
     }
     set.addInternal(rel);
     Set<String> variablesSet = RelOptUtil.getVariablesSet(rel);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
index 1389a36..4186232 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoPlanner.java
@@ -66,6 +66,7 @@ import org.apache.calcite.rel.rules.UnionToDistinctRule;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.runtime.Hook;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.SaffronProperties;
 import org.apache.calcite.util.Util;
@@ -168,7 +169,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   /**
    * List of all sets. Used only for debugging.
    */
-  final List<RelSet> allSets = new ArrayList<RelSet>();
+  final List<RelSet> allSets = new ArrayList<>();
 
   /**
    * Canonical map from {@link String digest} to the unique
@@ -180,7 +181,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * null VARCHAR(10).
    */
   private final Map<Pair<String, RelDataType>, RelNode> mapDigestToRel =
-      new HashMap<Pair<String, RelDataType>, RelNode>();
+      new HashMap<>();
 
   /**
    * Map each registered expression ({@link RelNode}) to its equivalence set
@@ -193,7 +194,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * to be careful, otherwise it gets incestuous.</p>
    */
   private final IdentityHashMap<RelNode, RelSubset> mapRel2Subset =
-      new IdentityHashMap<RelNode, RelSubset>();
+      new IdentityHashMap<>();
 
   /**
    * The importance of relational expressions.
@@ -205,13 +206,12 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * <p>If a RelNode has 0 importance, all {@link RelOptRuleCall}s using it
    * are ignored, and future RelOptRuleCalls are not queued up.
    */
-  final Map<RelNode, Double> relImportances = new HashMap<RelNode, Double>();
+  final Map<RelNode, Double> relImportances = new HashMap<>();
 
   /**
    * List of all schemas which have been registered.
    */
-  private final Set<RelOptSchema> registeredSchemas =
-      new HashSet<RelOptSchema>();
+  private final Set<RelOptSchema> registeredSchemas = new HashSet<>();
 
   /**
    * Holds rule calls waiting to be fired.
@@ -221,12 +221,12 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   /**
    * Holds the currently registered RelTraitDefs.
    */
-  private final List<RelTraitDef> traitDefs = new ArrayList<RelTraitDef>();
+  private final List<RelTraitDef> traitDefs = new ArrayList<>();
 
   /**
    * Set of all registered rules.
    */
-  protected final Set<RelOptRule> ruleSet = new HashSet<RelOptRule>();
+  protected final Set<RelOptRule> ruleSet = new HashSet<>();
 
   private int nextSetId = 0;
 
@@ -261,11 +261,9 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   private final Map<List<String>, RelOptLattice> latticeByName =
       Maps.newLinkedHashMap();
 
-  final Map<RelNode, Provenance> provenanceMap =
-      new HashMap<RelNode, Provenance>();
+  final Map<RelNode, Provenance> provenanceMap = new HashMap<>();
 
-  private final List<VolcanoRuleCall> ruleCallStack =
-      new ArrayList<VolcanoRuleCall>();
+  private final List<VolcanoRuleCall> ruleCallStack = new ArrayList<>();
 
   /** Zero cost, according to {@link #costFactory}. Not necessarily a
    * {@link org.apache.calcite.plan.volcano.VolcanoCost}. */
@@ -533,7 +531,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   }
 
   private static Set<RelOptTable> findTables(RelNode rel) {
-    final Set<RelOptTable> usedTables = new LinkedHashSet<RelOptTable>();
+    final Set<RelOptTable> usedTables = new LinkedHashSet<>();
     new RelVisitor() {
       @Override public void visit(RelNode node, int ordinal, RelNode parent) {
         if (node instanceof TableScan) {
@@ -896,7 +894,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
   private String provenance(RelNode root) {
     final StringWriter sw = new StringWriter();
     final PrintWriter pw = new PrintWriter(sw);
-    final List<RelNode> nodes = new ArrayList<RelNode>();
+    final List<RelNode> nodes = new ArrayList<>();
     new RelVisitor() {
       public void visit(RelNode node, int ordinal, RelNode parent) {
         nodes.add(node);
@@ -904,7 +902,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       }
       // CHECKSTYLE: IGNORE 1
     }.go(root);
-    final Set<RelNode> visited = new HashSet<RelNode>();
+    final Set<RelNode> visited = new HashSet<>();
     for (RelNode node : nodes) {
       provenanceRecurse(pw, node, 0, visited);
     }
@@ -953,7 +951,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
         new RelVisitor() {
           int depth = 0;
 
-          HashSet<RelSubset> visitedSubsets = new HashSet<RelSubset>();
+          final HashSet<RelSubset> visitedSubsets = new HashSet<>();
 
           public void visit(
               RelNode p,
@@ -993,7 +991,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
    * {@link Convention#NONE} and boosts their importance by 25%.
    */
   private void injectImportanceBoost() {
-    HashSet<RelSubset> requireBoost = new HashSet<RelSubset>();
+    final HashSet<RelSubset> requireBoost = new HashSet<>();
 
   SUBSET_LOOP:
     for (RelSubset subset : ruleQueue.subsetImportances.keySet()) {
@@ -1031,7 +1029,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           rel.getRowType(),
           "equivRel rowtype",
           equivRel.getRowType(),
-          true);
+          Litmus.THROW);
       set = getSet(equivRel);
     }
     final RelSubset subset = registerImpl(rel, set);
@@ -1688,7 +1686,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
           rel,
           new RuleProvenance(
               ruleCall.rule,
-              ImmutableList.<RelNode>copyOf(ruleCall.rels),
+              ImmutableList.copyOf(ruleCall.rels),
               ruleCall.id));
     }
 
@@ -1704,7 +1702,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       assert RelOptUtil.equal(
           "left", equivExp.getRowType(),
           "right", rel.getRowType(),
-          true);
+          Litmus.THROW);
       RelSet equivSet = getSet(equivExp);
       if (equivSet != null) {
         if (LOGGER.isLoggable(Level.FINER)) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
index 0e7facd..1938447 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -34,6 +34,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
@@ -196,8 +197,12 @@ public abstract class AbstractRelNode implements RelNode {
     return className;
   }
 
+  public boolean isValid(Litmus litmus) {
+    return litmus.succeed();
+  }
+
   public boolean isValid(boolean fail) {
-    return true;
+    return isValid(Litmus.THROW);
   }
 
   /** @deprecated Use {@link RelMetadataQuery#collations(RelNode)} */
@@ -310,7 +315,7 @@ public abstract class AbstractRelNode implements RelNode {
             input.getRowType(),
             "rowtype of rel after registration",
             e.getRowType(),
-            true);
+            Litmus.THROW);
       }
       inputs.add(e);
     }
@@ -319,7 +324,7 @@ public abstract class AbstractRelNode implements RelNode {
       r = copy(getTraitSet(), inputs);
     }
     r.recomputeDigest();
-    assert r.isValid(true);
+    assert r.isValid(Litmus.THROW);
     return r;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/RelNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/RelNode.java b/core/src/main/java/org/apache/calcite/rel/RelNode.java
index 7cf4de7..29dc023 100644
--- a/core/src/main/java/org/apache/calcite/rel/RelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/RelNode.java
@@ -29,6 +29,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 
 import java.util.List;
 import java.util.Set;
@@ -301,20 +302,23 @@ public interface RelNode extends RelOptNode, Cloneable {
    * Returns whether this relational expression is valid.
    *
    * <p>If assertions are enabled, this method is typically called with <code>
-   * fail</code> = <code>true</code>, as follows:
+   * litmus</code> = <code>THROW</code>, as follows:
    *
    * <blockquote>
-   * <pre>assert rel.isValid(true)</pre>
+   * <pre>assert rel.isValid(Litmus.THROW)</pre>
    * </blockquote>
    *
-   * This signals that the method can throw an {@link AssertionError} if it is
-   * not valid.
+   * <p>This signals that the method can throw an {@link AssertionError} if it
+   * is not valid.
    *
-   * @param fail Whether to fail if invalid
+   * @param litmus What to do if invalid
    * @return Whether relational expression is valid
    * @throws AssertionError if this relational expression is invalid and
-   *                        fail=true and assertions are enabled
+   *                        litmus is THROW
    */
+  boolean isValid(Litmus litmus);
+
+  @Deprecated // to be removed before 2.0
   boolean isValid(boolean fail);
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
index 1a46ffd..1a07bed 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Aggregate.java
@@ -40,6 +40,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.validate.SqlValidatorException;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -139,7 +140,7 @@ public abstract class Aggregate extends SingleRel {
     }
     assert groupSet.length() <= child.getRowType().getFieldCount();
     for (AggregateCall aggCall : aggCalls) {
-      assert typeMatchesInferred(aggCall, true);
+      assert typeMatchesInferred(aggCall, Litmus.THROW);
       Preconditions.checkArgument(aggCall.filterArg < 0
           || isPredicate(child, aggCall.filterArg),
           "filter must be BOOLEAN NOT NULL");
@@ -371,16 +372,9 @@ public abstract class Aggregate extends SingleRel {
     return builder.build();
   }
 
-  public boolean isValid(boolean fail) {
-    if (!super.isValid(fail)) {
-      assert !fail;
-      return false;
-    }
-    if (!Util.isDistinct(getRowType().getFieldNames())) {
-      assert !fail : getRowType();
-      return false;
-    }
-    return true;
+  public boolean isValid(Litmus litmus) {
+    return super.isValid(litmus)
+        && litmus.check(Util.isDistinct(getRowType().getFieldNames()), getRowType());
   }
 
   /**
@@ -388,12 +382,12 @@ public abstract class Aggregate extends SingleRel {
    * type it was given when it was created.
    *
    * @param aggCall Aggregate call
-   * @param fail    Whether to fail if the types do not match
+   * @param litmus What to do if an error is detected (types do not match)
    * @return Whether the inferred and declared types match
    */
   private boolean typeMatchesInferred(
       final AggregateCall aggCall,
-      final boolean fail) {
+      final Litmus litmus) {
     SqlAggFunction aggFunction = aggCall.getAggregation();
     AggCallBinding callBinding = aggCall.createBinding(this);
     RelDataType type = aggFunction.inferReturnType(callBinding);
@@ -402,7 +396,7 @@ public abstract class Aggregate extends SingleRel {
         expectedType,
         "inferred type",
         type,
-        fail);
+        litmus);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Calc.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Calc.java b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
index ea2f7d1..f4067eb 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Calc.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Calc.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.List;
@@ -63,7 +64,7 @@ public abstract class Calc extends SingleRel {
     super(cluster, traits, child);
     this.rowType = program.getOutputRowType();
     this.program = program;
-    assert isValid(true);
+    assert isValid(Litmus.THROW);
   }
 
   @Deprecated // to be removed before 2.0
@@ -109,22 +110,21 @@ public abstract class Calc extends SingleRel {
     return copy(traitSet, child, program);
   }
 
-  public boolean isValid(boolean fail) {
+  public boolean isValid(Litmus litmus) {
     if (!RelOptUtil.equal(
         "program's input type",
         program.getInputRowType(),
         "child's output type",
-        getInput().getRowType(),
-        fail)) {
-      return false;
+        getInput().getRowType(), litmus)) {
+      return litmus.fail(null);
     }
-    if (!program.isValid(fail)) {
-      return false;
+    if (!program.isValid(litmus)) {
+      return litmus.fail(null);
     }
-    if (!program.isNormalized(fail, getCluster().getRexBuilder())) {
-      return false;
+    if (!program.isNormalized(litmus, getCluster().getRexBuilder())) {
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RexProgram getProgram() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Filter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Filter.java b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
index 0dc3009..e5461e8 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Filter.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Filter.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexProgram;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableList;
 
@@ -73,7 +74,7 @@ public abstract class Filter extends SingleRel {
     assert RexUtil.isFlat(condition) : condition;
     this.condition = condition;
     // Too expensive for everyday use:
-    assert !CalcitePrepareImpl.DEBUG || isValid(true);
+    assert !CalcitePrepareImpl.DEBUG || isValid(Litmus.THROW);
   }
 
   /**
@@ -110,18 +111,16 @@ public abstract class Filter extends SingleRel {
     return condition;
   }
 
-  @Override public boolean isValid(boolean fail) {
+  @Override public boolean isValid(Litmus litmus) {
     if (RexUtil.isNullabilityCast(getCluster().getTypeFactory(), condition)) {
-      assert !fail : "Cast for just nullability not allowed";
-      return false;
+      return litmus.fail("Cast for just nullability not allowed");
     }
-    final RexChecker checker = new RexChecker(getInput().getRowType(), fail);
+    final RexChecker checker = new RexChecker(getInput().getRowType(), litmus);
     condition.accept(checker);
     if (checker.getFailureCount() > 0) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Join.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Join.java b/core/src/main/java/org/apache/calcite/rel/core/Join.java
index 5a0990d..bb460ff 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Join.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Join.java
@@ -32,6 +32,7 @@ import org.apache.calcite.rex.RexChecker;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.sql.type.SqlTypeName;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -117,22 +118,20 @@ public abstract class Join extends BiRel {
   }
 
   // TODO: enable
-  public boolean isValid_(boolean fail) {
-    if (!super.isValid(fail)) {
+  public boolean isValid_(Litmus litmus) {
+    if (!super.isValid(litmus)) {
       return false;
     }
     if (getRowType().getFieldCount()
         != getSystemFieldList().size()
         + left.getRowType().getFieldCount()
         + right.getRowType().getFieldCount()) {
-      assert !fail : "field count mismatch";
-      return false;
+      return litmus.fail("field count mismatch");
     }
     if (condition != null) {
       if (condition.getType().getSqlTypeName() != SqlTypeName.BOOLEAN) {
-        assert !fail
-            : "condition must be boolean: " + condition.getType();
-        return false;
+        return litmus.fail("condition must be boolean: "
+            + condition.getType());
       }
       // The input to the condition is a row type consisting of system
       // fields, left fields, and right fields. Very similar to the
@@ -145,16 +144,14 @@ public abstract class Join extends BiRel {
                   .addAll(getLeft().getRowType().getFieldList())
                   .addAll(getRight().getRowType().getFieldList())
                   .build(),
-              fail);
+              litmus);
       condition.accept(checker);
       if (checker.getFailureCount() > 0) {
-        assert !fail
-            : checker.getFailureCount() + " failures in condition "
-            + condition;
-        return false;
+        return litmus.fail(checker.getFailureCount()
+            + " failures in condition " + condition);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   @Override public RelOptCost computeSelfCost(RelOptPlanner planner) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Project.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Project.java b/core/src/main/java/org/apache/calcite/rel/core/Project.java
index 09c2ec9..86f4147 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Project.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Project.java
@@ -20,7 +20,6 @@ import org.apache.calcite.linq4j.Ord;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelOptCost;
 import org.apache.calcite.plan.RelOptPlanner;
-import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelInput;
 import org.apache.calcite.rel.RelNode;
@@ -30,14 +29,12 @@ import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.rex.RexChecker;
-import org.apache.calcite.rex.RexFieldAccess;
 import org.apache.calcite.rex.RexInputRef;
-import org.apache.calcite.rex.RexLocalRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.rex.RexShuttle;
 import org.apache.calcite.rex.RexUtil;
-import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.sql.SqlExplainLevel;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 import org.apache.calcite.util.Util;
@@ -82,7 +79,7 @@ public abstract class Project extends SingleRel {
     assert rowType != null;
     this.exps = ImmutableList.copyOf(projects);
     this.rowType = rowType;
-    assert isValid(true);
+    assert isValid(Litmus.THROW);
   }
 
   @Deprecated // to be removed before 2.0
@@ -174,31 +171,25 @@ public abstract class Project extends SingleRel {
     return 1;
   }
 
-  public boolean isValid(boolean fail) {
-    if (!super.isValid(fail)) {
-      assert !fail;
-      return false;
+  public boolean isValid(Litmus litmus) {
+    if (!super.isValid(litmus)) {
+      return litmus.fail(null);
     }
-    if (!RexUtil.compatibleTypes(
-        exps,
-        getRowType(),
-        true)) {
-      assert !fail;
-      return false;
+    if (!RexUtil.compatibleTypes(exps, getRowType(), litmus)) {
+      return litmus.fail("incompatible types");
     }
     RexChecker checker =
         new RexChecker(
-            getInput().getRowType(), fail);
+            getInput().getRowType(), litmus);
     for (RexNode exp : exps) {
       exp.accept(checker);
-    }
-    if (checker.getFailureCount() > 0) {
-      assert !fail;
-      return false;
+      if (checker.getFailureCount() > 0) {
+        return litmus.fail(checker.getFailureCount()
+            + " failures in expression " + exp);
+      }
     }
     if (!Util.isDistinct(rowType.getFieldNames())) {
-      assert !fail : rowType;
-      return false;
+      return litmus.fail("field names not distinct: " + rowType);
     }
     //CHECKSTYLE: IGNORE 1
     if (false && !Util.isDistinct(
@@ -214,10 +205,9 @@ public abstract class Project extends SingleRel {
       // because we need to allow
       //
       //  SELECT a, b FROM c UNION SELECT x, x FROM z
-      assert !fail : exps;
-      return false;
+      return litmus.fail("duplicate expressions: " + exps);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RelOptCost computeSelfCost(RelOptPlanner planner) {
@@ -374,80 +364,6 @@ public abstract class Project extends SingleRel {
     public static final int BOXED = 1;
     public static final int NONE = 0;
   }
-
-  /**
-   * Visitor which walks over a program and checks validity.
-   */
-  private static class Checker extends RexVisitorImpl<Boolean> {
-    private final boolean fail;
-    private final RelDataType inputRowType;
-    int failCount = 0;
-
-    /**
-     * Creates a Checker.
-     *
-     * @param inputRowType Input row type to expressions
-     * @param fail         Whether to throw if checker finds an error
-     */
-    private Checker(RelDataType inputRowType, boolean fail) {
-      super(true);
-      this.fail = fail;
-      this.inputRowType = inputRowType;
-    }
-
-    public Boolean visitInputRef(RexInputRef inputRef) {
-      final int index = inputRef.getIndex();
-      final List<RelDataTypeField> fields = inputRowType.getFieldList();
-      if ((index < 0) || (index >= fields.size())) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      if (!RelOptUtil.eq("inputRef",
-          inputRef.getType(),
-          "underlying field",
-          fields.get(index).getType(),
-          fail)) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      return true;
-    }
-
-    public Boolean visitLocalRef(RexLocalRef localRef) {
-      assert !fail : "localRef invalid in project";
-      ++failCount;
-      return false;
-    }
-
-    public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
-      super.visitFieldAccess(fieldAccess);
-      final RelDataType refType =
-          fieldAccess.getReferenceExpr().getType();
-      assert refType.isStruct();
-      final RelDataTypeField field = fieldAccess.getField();
-      final int index = field.getIndex();
-      if ((index < 0) || (index > refType.getFieldList().size())) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      final RelDataTypeField typeField =
-          refType.getFieldList().get(index);
-      if (!RelOptUtil.eq(
-          "type1",
-          typeField.getType(),
-          "type2",
-          fieldAccess.getType(),
-          fail)) {
-        assert !fail;
-        ++failCount;
-        return false;
-      }
-      return true;
-    }
-  }
 }
 
 // End Project.java

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/core/Window.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/core/Window.java b/core/src/main/java/org/apache/calcite/rel/core/Window.java
index 8db59cb..7efb115 100644
--- a/core/src/main/java/org/apache/calcite/rel/core/Window.java
+++ b/core/src/main/java/org/apache/calcite/rel/core/Window.java
@@ -40,6 +40,7 @@ import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.sql.SqlAggFunction;
 import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.ImmutableIntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
@@ -84,7 +85,7 @@ public abstract class Window extends SingleRel {
     this.groups = ImmutableList.copyOf(groups);
   }
 
-  @Override public boolean isValid(boolean fail) {
+  @Override public boolean isValid(Litmus litmus) {
     // In the window specifications, an aggregate call such as
     // 'SUM(RexInputRef #10)' refers to expression #10 of inputProgram.
     // (Not its projections.)
@@ -106,21 +107,20 @@ public abstract class Window extends SingleRel {
         };
 
     final RexChecker checker =
-        new RexChecker(inputTypes, fail);
+        new RexChecker(inputTypes, litmus);
     int count = 0;
     for (Group group : groups) {
       for (RexWinAggCall over : group.aggCalls) {
         ++count;
         if (!checker.isValid(over)) {
-          return false;
+          return litmus.fail(null);
         }
       }
     }
     if (count == 0) {
-      assert !fail : "empty";
-      return false;
+      return litmus.fail("empty");
     }
-    return true;
+    return litmus.succeed();
   }
 
   public RelWriter explainTerms(RelWriter pw) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
index 559dc0e..783a649 100644
--- a/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
+++ b/core/src/main/java/org/apache/calcite/rel/logical/LogicalWindow.java
@@ -36,6 +36,7 @@ import org.apache.calcite.rex.RexWindow;
 import org.apache.calcite.rex.RexWindowBound;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableBitSet;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Objects;
@@ -222,7 +223,7 @@ public final class LogicalWindow extends Window {
                 over.getType(),
                 "aggCall",
                 aggCall.getType(),
-                true);
+                Litmus.THROW);
 
             // Find the index of the aggCall among all partitions of all
             // groups.
@@ -237,7 +238,7 @@ public final class LogicalWindow extends Window {
                 over.getType(),
                 "intermed",
                 intermediateRowType.getFieldList().get(index).getType(),
-                true);
+                Litmus.THROW);
             return new RexInputRef(
                 index,
                 over.getType());

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java b/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
index 047e950..2e4e9e0 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/CalcRelSplitter.java
@@ -37,6 +37,7 @@ import org.apache.calcite.rex.RexUtil;
 import org.apache.calcite.rex.RexVisitorImpl;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.IntList;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
@@ -121,7 +122,7 @@ public abstract class CalcRelSplitter {
     // expression is trivial (either an atom, or a function applied to
     // references to atoms) and every expression depends only on
     // expressions to the left.
-    assert program.isValid(true);
+    assert program.isValid(Litmus.THROW);
     final List<RexNode> exprList = program.getExprList();
     final RexNode[] exprs = exprList.toArray(new RexNode[exprList.size()]);
     assert !RexUtil.containComplexExprs(exprList);

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexCall.java b/core/src/main/java/org/apache/calcite/rex/RexCall.java
index b06ffc9..8271005 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexCall.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexCall.java
@@ -20,6 +20,7 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.SqlSyntax;
+import org.apache.calcite.util.Litmus;
 
 import com.google.common.collect.ImmutableList;
 
@@ -63,7 +64,7 @@ public class RexCall extends RexNode {
     assert op.getKind() != null : op;
     this.digest = computeDigest(true);
 
-    assert op.validRexOperands(operands.size(), true) : this;
+    assert op.validRexOperands(operands.size(), Litmus.THROW) : this;
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexChecker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexChecker.java b/core/src/main/java/org/apache/calcite/rex/RexChecker.java
index 77edc1f..c51d923 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexChecker.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexChecker.java
@@ -19,6 +19,7 @@ package org.apache.calcite.rex;
 import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
+import org.apache.calcite.util.Litmus;
 
 import java.util.List;
 
@@ -55,7 +56,7 @@ import java.util.List;
 public class RexChecker extends RexVisitorImpl<Boolean> {
   //~ Instance fields --------------------------------------------------------
 
-  protected final boolean fail;
+  protected final Litmus litmus;
   protected final List<RelDataType> inputTypeList;
   protected int failCount;
 
@@ -71,11 +72,10 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
    * <p>Otherwise, each method returns whether its part of the tree is valid.
    *
    * @param inputRowType Input row type
-   * @param fail Whether to throw an {@link AssertionError} if an
-   *                     invalid node is detected
+   * @param litmus What to do if an invalid node is detected
    */
-  public RexChecker(final RelDataType inputRowType, boolean fail) {
-    this(RelOptUtil.getFieldTypeList(inputRowType), fail);
+  public RexChecker(final RelDataType inputRowType, Litmus litmus) {
+    this(RelOptUtil.getFieldTypeList(inputRowType), litmus);
   }
 
   /**
@@ -88,13 +88,12 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
    * <p>Otherwise, each method returns whether its part of the tree is valid.
    *
    * @param inputTypeList Input row type
-   * @param fail Whether to throw an {@link AssertionError} if an
-   *                      invalid node is detected
+   * @param litmus What to do if an error is detected
    */
-  public RexChecker(List<RelDataType> inputTypeList, boolean fail) {
+  public RexChecker(List<RelDataType> inputTypeList, Litmus litmus) {
     super(true);
     this.inputTypeList = inputTypeList;
-    this.fail = fail;
+    this.litmus = litmus;
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -111,36 +110,32 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
   public Boolean visitInputRef(RexInputRef ref) {
     final int index = ref.getIndex();
     if ((index < 0) || (index >= inputTypeList.size())) {
-      assert !fail
-          : "RexInputRef index " + index
-          + " out of range 0.." + (inputTypeList.size() - 1);
       ++failCount;
-      return false;
+      return litmus.fail("RexInputRef index " + index
+          + " out of range 0.." + (inputTypeList.size() - 1));
     }
     if (!ref.getType().isStruct()
         && !RelOptUtil.eq("ref", ref.getType(), "input",
-            inputTypeList.get(index), fail)) {
-      assert !fail;
+            inputTypeList.get(index), litmus)) {
       ++failCount;
-      return false;
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public Boolean visitLocalRef(RexLocalRef ref) {
-    assert !fail : "RexLocalRef illegal outside program";
     ++failCount;
-    return false;
+    return litmus.fail("RexLocalRef illegal outside program");
   }
 
   public Boolean visitCall(RexCall call) {
     for (RexNode operand : call.getOperands()) {
       Boolean valid = operand.accept(this);
       if (valid != null && !valid) {
-        return false;
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   public Boolean visitFieldAccess(RexFieldAccess fieldAccess) {
@@ -150,22 +145,19 @@ public class RexChecker extends RexVisitorImpl<Boolean> {
     final RelDataTypeField field = fieldAccess.getField();
     final int index = field.getIndex();
     if ((index < 0) || (index > refType.getFieldList().size())) {
-      assert !fail;
       ++failCount;
-      return false;
+      return litmus.fail(null);
     }
     final RelDataTypeField typeField = refType.getFieldList().get(index);
     if (!RelOptUtil.eq(
         "type1",
         typeField.getType(),
         "type2",
-        fieldAccess.getType(),
-        fail)) {
-      assert !fail;
+        fieldAccess.getType(), litmus)) {
       ++failCount;
-      return false;
+      return litmus.fail(null);
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexProgram.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgram.java b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
index 6fd12ce..58cdc40 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgram.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgram.java
@@ -27,6 +27,7 @@ import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.SqlExplainLevel;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeUtil;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Permutation;
 
@@ -110,7 +111,7 @@ public class RexProgram {
     this.projects = ImmutableList.copyOf(projects);
     this.condition = condition;
     this.outputRowType = outputRowType;
-    assert isValid(true);
+    assert isValid(Litmus.THROW);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -394,64 +395,53 @@ public class RexProgram {
    * will throw an {@link AssertionError} if assertions are enabled. If <code>
    * fail</code> is false, merely returns whether the program is valid.
    *
-   * @param fail Whether to fail
+   * @param litmus What to do if an error is detected
    * @return Whether the program is valid
-   * @throws AssertionError if program is invalid and <code>fail</code> is
-   *                        true and assertions are enabled
    */
-  public boolean isValid(boolean fail) {
+  public boolean isValid(Litmus litmus) {
     if (inputRowType == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     if (exprs == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     if (projects == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     if (outputRowType == null) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
 
     // If the input row type is a struct (contains fields) then the leading
     // expressions must be references to those fields. But we don't require
     // this if the input row type is, say, a java class.
     if (inputRowType.isStruct()) {
-      if (!RexUtil.containIdentity(exprs, inputRowType, fail)) {
-        assert !fail;
-        return false;
+      if (!RexUtil.containIdentity(exprs, inputRowType, litmus)) {
+        return litmus.fail(null);
       }
 
       // None of the other fields should be inputRefs.
       for (int i = inputRowType.getFieldCount(); i < exprs.size(); i++) {
         RexNode expr = exprs.get(i);
         if (expr instanceof RexInputRef) {
-          assert !fail;
-          return false;
+          return litmus.fail(null);
         }
       }
     }
     // todo: enable
     // CHECKSTYLE: IGNORE 1
-    if (false && RexUtil.containCommonExprs(exprs, fail)) {
-      assert !fail;
-      return false;
+    if (false && RexUtil.containNoCommonExprs(exprs, litmus)) {
+      return litmus.fail(null);
     }
-    if (RexUtil.containForwardRefs(exprs, inputRowType, fail)) {
-      assert !fail;
-      return false;
+    if (!RexUtil.containNoForwardRefs(exprs, inputRowType, litmus)) {
+      return litmus.fail(null);
     }
-    if (RexUtil.containNonTrivialAggs(exprs, fail)) {
-      assert !fail;
-      return false;
+    if (!RexUtil.containNoNonTrivialAggs(exprs, litmus)) {
+      return litmus.fail(null);
     }
     final Checker checker =
         new Checker(
-            fail,
+            litmus,
             inputRowType,
             new AbstractList<RelDataType>() {
               public RelDataType get(int index) {
@@ -465,30 +455,26 @@ public class RexProgram {
             });
     if (condition != null) {
       if (!SqlTypeUtil.inBooleanFamily(condition.getType())) {
-        assert !fail : "condition must be boolean";
-        return false;
+        return litmus.fail("condition must be boolean");
       }
       condition.accept(checker);
       if (checker.failCount > 0) {
-        assert !fail;
-        return false;
+        return litmus.fail(null);
       }
     }
     for (int i = 0; i < projects.size(); i++) {
       projects.get(i).accept(checker);
       if (checker.failCount > 0) {
-        assert !fail;
-        return false;
+        return litmus.fail(null);
       }
     }
     for (int i = 0; i < exprs.size(); i++) {
       exprs.get(i).accept(checker);
       if (checker.failCount > 0) {
-        assert !fail;
-        return false;
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**
@@ -773,24 +759,39 @@ public class RexProgram {
   /**
    * Returns whether this program is in canonical form.
    *
-   * @param fail       Whether to throw an assertion error if not in canonical
-   *                   form
+   * @param litmus     What to do if an error is detected (program is not in
+   *                   canonical form)
    * @param rexBuilder Rex builder
    * @return whether in canonical form
    */
-  public boolean isNormalized(boolean fail, RexBuilder rexBuilder) {
-    final RexProgram normalizedProgram =
-        RexProgramBuilder.normalize(rexBuilder, this);
+  public boolean isNormalized(Litmus litmus, RexBuilder rexBuilder) {
+    final RexProgram normalizedProgram = normalize(rexBuilder, false);
     String normalized = normalizedProgram.toString();
     String string = toString();
     if (!normalized.equals(string)) {
-      assert !fail
-          : "Program is not normalized:\n"
+      return litmus.fail("Program is not normalized:\n"
           + "program:    " + string + "\n"
-          + "normalized: " + normalized + "\n";
-      return false;
+          + "normalized: " + normalized + "\n");
     }
-    return true;
+    return litmus.succeed();
+  }
+
+  /**
+   * Creates a simplified/normalized copy of this program.
+   *
+   * @param rexBuilder Rex builder
+   * @param simplify Whether to simplify (in addition to normalizing)
+   * @return Normalized program
+   */
+  public RexProgram normalize(RexBuilder rexBuilder, boolean simplify) {
+    // Normalize program by creating program builder from the program, then
+    // converting to a program. getProgram does not need to normalize
+    // because the builder was normalized on creation.
+    assert isValid(Litmus.THROW);
+    final RexProgramBuilder builder =
+        RexProgramBuilder.create(rexBuilder, inputRowType, exprs, projects,
+            condition, outputRowType, simplify);
+    return builder.getProgram(false);
   }
 
   //~ Inner Classes ----------------------------------------------------------
@@ -804,15 +805,14 @@ public class RexProgram {
     /**
      * Creates a Checker.
      *
-     * @param fail                 Whether to fail
+     * @param litmus               Whether to fail
      * @param inputRowType         Types of the input fields
      * @param internalExprTypeList Types of the internal expressions
      */
-    public Checker(
-        boolean fail,
+    public Checker(Litmus litmus,
         RelDataType inputRowType,
         List<RelDataType> internalExprTypeList) {
-      super(inputRowType, fail);
+      super(inputRowType, litmus);
       this.internalExprTypeList = internalExprTypeList;
     }
 
@@ -821,21 +821,18 @@ public class RexProgram {
     public Boolean visitLocalRef(RexLocalRef localRef) {
       final int index = localRef.getIndex();
       if ((index < 0) || (index >= internalExprTypeList.size())) {
-        assert !fail;
         ++failCount;
-        return false;
+        return litmus.fail(null);
       }
       if (!RelOptUtil.eq(
           "type1",
           localRef.getType(),
           "type2",
-          internalExprTypeList.get(index),
-          fail)) {
-        assert !fail;
+          internalExprTypeList.get(index), litmus)) {
         ++failCount;
-        return false;
+        return litmus.fail(null);
       }
-      return true;
+      return litmus.succeed();
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
index bc89347..c292e56 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -20,6 +20,7 @@ import org.apache.calcite.plan.RelOptUtil;
 import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.rel.type.RelDataTypeField;
 import org.apache.calcite.sql.fun.SqlStdOperatorTable;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
@@ -449,7 +450,7 @@ public class RexProgramBuilder {
       RexProgram program,
       RexBuilder rexBuilder,
       boolean normalize) {
-    assert program.isValid(true);
+    assert program.isValid(Litmus.THROW);
     final RelDataType inputRowType = program.getInputRowType();
     final List<RexLocalRef> projectRefs = program.getProjectList();
     final RexLocalRef conditionRef = program.getCondition();
@@ -726,8 +727,8 @@ public class RexProgramBuilder {
       boolean normalize) {
     // Initialize a program builder with the same expressions, outputs
     // and condition as the bottom program.
-    assert bottomProgram.isValid(true);
-    assert topProgram.isValid(true);
+    assert bottomProgram.isValid(Litmus.THROW);
+    assert topProgram.isValid(Litmus.THROW);
     final RexProgramBuilder progBuilder =
         RexProgramBuilder.forProgram(bottomProgram, rexBuilder, false);
 
@@ -745,7 +746,7 @@ public class RexProgramBuilder {
       progBuilder.addProject(pair.left, pair.right);
     }
     RexProgram mergedProg = progBuilder.getProgram(normalize);
-    assert mergedProg.isValid(true);
+    assert mergedProg.isValid(Litmus.THROW);
     assert mergedProg.getOutputRowType() == topProgram.getOutputRowType();
     return mergedProg;
   }
@@ -906,7 +907,7 @@ public class RexProgramBuilder {
         assert input.getType().isStruct()
             || RelOptUtil.eq("type1", input.getType(),
                 "type2", inputRowType.getFieldList().get(index).getType(),
-                true);
+                Litmus.THROW);
       }
 
       // Return a reference to the N'th expression, which should be
@@ -927,7 +928,7 @@ public class RexProgramBuilder {
             exprList.get(index).getType(),
             "ref type",
             local.getType(),
-            true);
+            Litmus.THROW);
       }
 
       // Resolve the expression to an input.
@@ -994,7 +995,7 @@ public class RexProgramBuilder {
           local.getType(),
           "type2",
           input.getType(),
-          true);
+          Litmus.THROW);
       return local;
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/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 d801d1b..8e8a79c 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexUtil.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexUtil.java
@@ -33,6 +33,7 @@ import org.apache.calcite.sql.fun.SqlStdOperatorTable;
 import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.util.ControlFlowException;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.mapping.Mappings;
@@ -476,34 +477,35 @@ public class RexUtil {
   /**
    * Returns whether an array of expressions has any common sub-expressions.
    */
-  public static boolean containCommonExprs(List<RexNode> exprs, boolean fail) {
+  public static boolean containNoCommonExprs(List<RexNode> exprs,
+      Litmus litmus) {
     final ExpressionNormalizer visitor = new ExpressionNormalizer(false);
     for (RexNode expr : exprs) {
       try {
         expr.accept(visitor);
       } catch (ExpressionNormalizer.SubExprExistsException e) {
         Util.swallow(e, null);
-        assert !fail;
-        return true;
+        return litmus.fail(null);
       }
     }
-    return false;
+    return litmus.succeed();
   }
 
   /**
-   * Returns whether an array of expressions contains a forward reference.
+   * Returns whether an array of expressions contains no forward references.
    * That is, if expression #i contains a {@link RexInputRef} referencing
    * field i or greater.
    *
    * @param exprs        Array of expressions
    * @param inputRowType Input row type
-   * @param fail         Whether to assert if there is a forward reference
+   * @param litmus       What to do if an error is detected (there is a
+   *                     forward reference)
+   *
    * @return Whether there is a forward reference
    */
-  public static boolean containForwardRefs(
-      List<RexNode> exprs,
+  public static boolean containNoForwardRefs(List<RexNode> exprs,
       RelDataType inputRowType,
-      boolean fail) {
+      Litmus litmus) {
     final ForwardRefFinder visitor = new ForwardRefFinder(inputRowType);
     for (int i = 0; i < exprs.size(); i++) {
       RexNode expr = exprs.get(i);
@@ -512,21 +514,20 @@ public class RexUtil {
         expr.accept(visitor);
       } catch (ForwardRefFinder.IllegalForwardRefException e) {
         Util.swallow(e, null);
-        assert !fail : "illegal forward reference in " + expr;
-        return true;
+        return litmus.fail("illegal forward reference in " + expr);
       }
     }
-    return false;
+    return litmus.succeed();
   }
 
   /**
-   * Returns whether an array of exp contains aggregate function calls whose
-   * arguments are not {@link RexInputRef}.s
+   * Returns whether an array of exp contains no aggregate function calls whose
+   * arguments are not {@link RexInputRef}s.
    *
    * @param exprs Expressions
-   * @param fail  Whether to assert if there is such a function call
+   * @param litmus  Whether to assert if there is such a function call
    */
-  static boolean containNonTrivialAggs(List<RexNode> exprs, boolean fail) {
+  static boolean containNoNonTrivialAggs(List<RexNode> exprs, Litmus litmus) {
     for (RexNode expr : exprs) {
       if (expr instanceof RexCall) {
         RexCall rexCall = (RexCall) expr;
@@ -534,14 +535,13 @@ public class RexUtil {
           for (RexNode operand : rexCall.operands) {
             if (!(operand instanceof RexLocalRef)
                 && !(operand instanceof RexLiteral)) {
-              assert !fail : "contains non trivial agg: " + operand;
-              return true;
+              return litmus.fail("contains non trivial agg: " + operand);
             }
           }
         }
       }
     }
-    return false;
+    return litmus.succeed();
   }
 
   /**
@@ -623,28 +623,29 @@ public class RexUtil {
    *
    * @param exprs Array of expressions
    * @param type  Type
-   * @param fail  Whether to fail if there is a mismatch
+   * @param litmus What to do if an error is detected (there is a mismatch)
+   *
    * @return Whether every expression has the same type as the corresponding
    * member of the struct type
-   * @see RelOptUtil#eq(String, RelDataType, String, RelDataType, boolean)
+   *
+   * @see RelOptUtil#eq(String, RelDataType, String, RelDataType, org.apache.calcite.util.Litmus)
    */
   public static boolean compatibleTypes(
       List<RexNode> exprs,
       RelDataType type,
-      boolean fail) {
+      Litmus litmus) {
     final List<RelDataTypeField> fields = type.getFieldList();
     if (exprs.size() != fields.size()) {
-      assert !fail : "rowtype mismatches expressions";
-      return false;
+      return litmus.fail("rowtype mismatches expressions");
     }
     for (int i = 0; i < fields.size(); i++) {
       final RelDataType exprType = exprs.get(i).getType();
       final RelDataType fieldType = fields.get(i).getType();
-      if (!RelOptUtil.eq("type1", exprType, "type2", fieldType, fail)) {
-        return false;
+      if (!RelOptUtil.eq("type1", exprType, "type2", fieldType, litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**
@@ -665,39 +666,35 @@ public class RexUtil {
   public static boolean containIdentity(
       List<? extends RexNode> exprs,
       RelDataType rowType,
-      boolean fail) {
+      Litmus litmus) {
     final List<RelDataTypeField> fields = rowType.getFieldList();
     if (exprs.size() < fields.size()) {
-      assert !fail : "exprs/rowType length mismatch";
-      return false;
+      return litmus.fail("exprs/rowType length mismatch");
     }
     for (int i = 0; i < fields.size(); i++) {
       if (!(exprs.get(i) instanceof RexInputRef)) {
-        assert !fail : "expr[" + i + "] is not a RexInputRef";
-        return false;
+        return litmus.fail("expr[" + i + "] is not a RexInputRef");
       }
       RexInputRef inputRef = (RexInputRef) exprs.get(i);
       if (inputRef.getIndex() != i) {
-        assert !fail : "expr[" + i + "] has ordinal "
-            + inputRef.getIndex();
-        return false;
+        return litmus.fail("expr[" + i + "] has ordinal "
+            + inputRef.getIndex());
       }
       if (!RelOptUtil.eq("type1",
           exprs.get(i).getType(),
           "type2",
-          fields.get(i).getType(),
-          fail)) {
-        return false;
+          fields.get(i).getType(), litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   /** Returns whether a list of expressions projects the incoming fields. */
   public static boolean isIdentity(List<? extends RexNode> exps,
       RelDataType inputRowType) {
     return inputRowType.getFieldCount() == exps.size()
-        && containIdentity(exps, inputRowType, false);
+        && containIdentity(exps, inputRowType, Litmus.IGNORE);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
index 58b0d52..8d5f380 100644
--- a/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
+++ b/core/src/main/java/org/apache/calcite/schema/SchemaPlus.java
@@ -30,7 +30,16 @@ import com.google.common.collect.ImmutableList;
  *
  * <p>A user-defined schema does not need to implement this interface, but by
  * the time a schema is passed to a method in a user-defined schema or
- * user-defined table, it will have been wrapped in this interface.</p>
+ * user-defined table, it will have been wrapped in this interface.
+ *
+ * <p>SchemaPlus is intended to be used by users but not instantiated by them.
+ * Users should only use the SchemaPlus they are given by the system.
+ * The purpose of SchemaPlus is to expose to user code, in a read only manner,
+ * some of the extra information about schemas that Calcite builds up when a
+ * schema is registered. It appears in several SPI calls as context; for example
+ * {@link SchemaFactory#create(SchemaPlus, String, java.util.Map)} contains a
+ * parent schema that might be a wrapped instance of a user-defined
+ * {@link Schema}, or indeed might not.
  */
 public interface SchemaPlus extends Schema {
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
index c5f9f66..34943c2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlBinaryOperator.java
@@ -25,6 +25,7 @@ import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.math.BigDecimal;
@@ -208,7 +209,7 @@ public class SqlBinaryOperator extends SqlOperator {
     return super.getMonotonicity(call);
   }
 
-  @Override public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
     if (count != 2) {
       // Special exception for AND and OR.
       if ((this == SqlStdOperatorTable.AND
@@ -216,10 +217,9 @@ public class SqlBinaryOperator extends SqlOperator {
           && count > 2) {
         return true;
       }
-      assert !fail : "wrong operand count " + count + " for " + this;
-      return false;
+      return litmus.fail("wrong operand count " + count + " for " + this);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCall.java b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
index 3461b2d..59066b3 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCall.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCall.java
@@ -24,6 +24,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorImpl;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -134,23 +135,21 @@ public abstract class SqlCall extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (node == this) {
       return true;
     }
     if (!(node instanceof SqlCall)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlCall that = (SqlCall) node;
 
     // Compare operators by name, not identity, because they may not
     // have been resolved yet.
     if (!this.getOperator().getName().equals(that.getOperator().getName())) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return equalDeep(this.getOperandList(), that.getOperandList(), fail);
+    return equalDeep(this.getOperandList(), that.getOperandList(), litmus);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
index 62abaa6..8ce7923 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDataTypeSpec.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Objects;
@@ -234,39 +235,33 @@ public class SqlDataTypeSpec extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlDataTypeSpec)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlDataTypeSpec that = (SqlDataTypeSpec) node;
     if (!SqlNode.equalDeep(
         this.collectionsTypeName,
-        that.collectionsTypeName,
-        fail)) {
-      return false;
+        that.collectionsTypeName, litmus)) {
+      return litmus.fail(null);
     }
-    if (!this.typeName.equalsDeep(that.typeName, fail)) {
-      return false;
+    if (!this.typeName.equalsDeep(that.typeName, litmus)) {
+      return litmus.fail(null);
     }
     if (this.precision != that.precision) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     if (this.scale != that.scale) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     if (!Objects.equal(this.timeZone, that.timeZone)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     if (!com.google.common.base.Objects.equal(this.charSetName,
         that.charSetName)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
index 552852c..8fc771c 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlDynamicParam.java
@@ -21,6 +21,7 @@ import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 /**
  * A <code>SqlDynamicParam</code> represents a dynamic parameter marker in an
@@ -76,17 +77,15 @@ public class SqlDynamicParam extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlDynamicParam)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlDynamicParam that = (SqlDynamicParam) node;
     if (this.index != that.index) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
index 9d0d633..bd0c5d5 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIdentifier.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlQualified;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
@@ -286,23 +287,20 @@ public class SqlIdentifier extends SqlNode {
     validator.validateIdentifier(this, scope);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlIdentifier)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlIdentifier that = (SqlIdentifier) node;
     if (this.names.size() != that.names.size()) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     for (int i = 0; i < names.size(); i++) {
       if (!this.names.get(i).equals(that.names.get(i))) {
-        assert !fail : this + "!=" + node;
-        return false;
+        return litmus.fail(this + "!=" + node);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   public <R> R accept(SqlVisitor<R> visitor) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
index 089cbf6..aa9729d 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlIntervalQualifier.java
@@ -26,6 +26,7 @@ import org.apache.calcite.sql.type.SqlTypeName;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
@@ -142,14 +143,13 @@ public class SqlIntervalQualifier extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     final String thisString = this.toString();
     final String thatString = node.toString();
     if (!thisString.equals(thatString)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public int getStartPrecision(RelDataTypeSystem typeSystem) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
index 3295fb5..32714c6 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -31,6 +31,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.util.BitString;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Util;
 
@@ -403,17 +404,15 @@ public class SqlLiteral extends SqlNode {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlLiteral)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlLiteral that = (SqlLiteral) node;
     if (!this.equals(that)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
-    return true;
+    return litmus.succeed();
   }
 
   public SqlMonotonicity getMonotonicity(SqlValidatorScope scope) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNode.java b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
index 6531114..b31cf7b 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -24,6 +24,7 @@ import org.apache.calcite.sql.validate.SqlMoniker;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.Collection;
@@ -254,27 +255,32 @@ public abstract class SqlNode implements Cloneable {
    * (2 + 3), because the '+' operator is left-associative</li>
    * </ul>
    */
-  public abstract boolean equalsDeep(SqlNode node, boolean fail);
+  public abstract boolean equalsDeep(SqlNode node, Litmus litmus);
+
+  @Deprecated // to be removed before 2.0
+  public final boolean equalsDeep(SqlNode node, boolean fail) {
+    return equalsDeep(node, fail ? Litmus.THROW : Litmus.IGNORE);
+  }
 
   /**
    * Returns whether two nodes are equal (using
-   * {@link #equalsDeep(SqlNode, boolean)}) or are both null.
+   * {@link #equalsDeep(SqlNode, Litmus)}) or are both null.
    *
    * @param node1 First expression
    * @param node2 Second expression
-   * @param fail  Whether to throw {@link AssertionError} if expressions are
-   *              not equal
+   * @param litmus What to do if an error is detected (expressions are
+   *              not equal)
    */
   public static boolean equalDeep(
       SqlNode node1,
       SqlNode node2,
-      boolean fail) {
+      Litmus litmus) {
     if (node1 == null) {
       return node2 == null;
     } else if (node2 == null) {
       return false;
     } else {
-      return node1.equalsDeep(node2, fail);
+      return node1.equalsDeep(node2, litmus);
     }
   }
 
@@ -294,17 +300,16 @@ public abstract class SqlNode implements Cloneable {
 
   /** Returns whether two lists of operands are equal. */
   public static boolean equalDeep(List<SqlNode> operands0,
-      List<SqlNode> operands1, boolean fail) {
+      List<SqlNode> operands1, Litmus litmus) {
     if (operands0.size() != operands1.size()) {
-      assert !fail;
-      return false;
+      return litmus.fail(null);
     }
     for (int i = 0; i < operands0.size(); i++) {
-      if (!SqlNode.equalDeep(operands0.get(i), operands1.get(i), fail)) {
-        return false;
+      if (!SqlNode.equalDeep(operands0.get(i), operands1.get(i), litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
index 4d77dac..392d967 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNodeList.java
@@ -21,6 +21,7 @@ import org.apache.calcite.sql.parser.SqlParserPos;
 import org.apache.calcite.sql.util.SqlVisitor;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
+import org.apache.calcite.util.Litmus;
 
 import java.util.ArrayList;
 import java.util.Collection;
@@ -55,7 +56,7 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
    */
   public SqlNodeList(SqlParserPos pos) {
     super(pos);
-    list = new ArrayList<SqlNode>();
+    list = new ArrayList<>();
   }
 
   /**
@@ -66,7 +67,7 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
       Collection<? extends SqlNode> collection,
       SqlParserPos pos) {
     super(pos);
-    list = new ArrayList<SqlNode>(collection);
+    list = new ArrayList<>(collection);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -152,24 +153,22 @@ public class SqlNodeList extends SqlNode implements Iterable<SqlNode> {
     return visitor.visit(this);
   }
 
-  public boolean equalsDeep(SqlNode node, boolean fail) {
+  public boolean equalsDeep(SqlNode node, Litmus litmus) {
     if (!(node instanceof SqlNodeList)) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     SqlNodeList that = (SqlNodeList) node;
     if (this.size() != that.size()) {
-      assert !fail : this + "!=" + node;
-      return false;
+      return litmus.fail(this + "!=" + node);
     }
     for (int i = 0; i < list.size(); i++) {
       SqlNode thisChild = list.get(i);
       final SqlNode thatChild = that.list.get(i);
-      if (!thisChild.equalsDeep(thatChild, fail)) {
-        return false;
+      if (!thisChild.equalsDeep(thatChild, litmus)) {
+        return litmus.fail(null);
       }
     }
-    return true;
+    return litmus.succeed();
   }
 
   public SqlNode[] toArray() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
index df165ca..bef8b59 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlOperator.java
@@ -29,6 +29,7 @@ import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 import java.util.Arrays;
@@ -590,7 +591,7 @@ public abstract class SqlOperator {
    * (some examples are CAST and AND), and this method throws internal errors,
    * not user errors.</p>
    */
-  public boolean validRexOperands(int count, boolean fail) {
+  public boolean validRexOperands(int count, Litmus litmus) {
     return true;
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
index 9b36a08..5fef8ae 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.type.SqlOperandTypeInference;
 import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 /**
@@ -88,12 +89,11 @@ public class SqlPostfixOperator extends SqlOperator {
     return type;
   }
 
-  @Override public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
     if (count != 1) {
-      assert !fail : "wrong operand count " + count + " for " + this;
-      return false;
+      return litmus.fail("wrong operand count " + count + " for " + this);
     }
-    return true;
+    return litmus.succeed();
   }
 }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/9a52b5ed/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java b/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
index 8923645..8976a56 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
@@ -23,6 +23,7 @@ import org.apache.calcite.sql.type.SqlReturnTypeInference;
 import org.apache.calcite.sql.type.SqlTypeUtil;
 import org.apache.calcite.sql.validate.SqlMonotonicity;
 import org.apache.calcite.sql.validate.SqlValidator;
+import org.apache.calcite.util.Litmus;
 import org.apache.calcite.util.Util;
 
 /**
@@ -97,12 +98,11 @@ public class SqlPrefixOperator extends SqlOperator {
     return super.getMonotonicity(call);
   }
 
-  @Override public boolean validRexOperands(int count, boolean fail) {
+  @Override public boolean validRexOperands(int count, Litmus litmus) {
     if (count != 1) {
-      assert !fail : "wrong operand count " + count + " for " + this;
-      return false;
+      return litmus.fail("wrong operand count " + count + " for " + this);
     }
-    return true;
+    return litmus.succeed();
   }
 }