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 2017/01/25 21:25:17 UTC

[3/9] calcite git commit: [CALCITE-1597] Obsolete Util.newInternal, .pre, .post, .permAssert and Throwables.propagate

[CALCITE-1597] Obsolete Util.newInternal, .pre, .post, .permAssert and Throwables.propagate

Replace Util.newInternal with new AssertionError in some contexts,
new RuntimeException in others. Replace methods pre, pos, permAssert
in Util with methods in Preconditions.

Remove uses of Throwables.propagate and .propagateIfPossible because
they are deprecated as of Guava 20. Create method
Util.throwIfUnchecked because we can't use Guava's
Throwables.throwIfUnchecked yet.


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

Branch: refs/heads/master
Commit: 603648bc2466ab699fa6646ffc1ad717b8c11880
Parents: 89f1125
Author: Julian Hyde <jh...@apache.org>
Authored: Mon Jan 23 01:38:12 2017 -0800
Committer: Julian Hyde <jh...@apache.org>
Committed: Wed Jan 25 11:00:12 2017 -0800

----------------------------------------------------------------------
 .../adapter/cassandra/CassandraRules.java       |  13 +--
 core/src/main/codegen/templates/Parser.jj       |   4 +-
 .../calcite/interpreter/AggregateNode.java      |   9 +-
 .../calcite/jdbc/CalciteConnectionImpl.java     |   5 +-
 .../CachingLatticeStatisticProvider.java        |   5 +-
 .../apache/calcite/plan/ConventionTraitDef.java |   4 +-
 .../org/apache/calcite/plan/RelOptUtil.java     |  13 ++-
 .../org/apache/calcite/plan/hep/HepPlanner.java |   5 +-
 .../calcite/plan/volcano/VolcanoPlanner.java    |  27 ++---
 .../calcite/plan/volcano/VolcanoRuleCall.java   |  15 ++-
 .../org/apache/calcite/rel/AbstractRelNode.java |   2 +-
 .../main/java/org/apache/calcite/rel/BiRel.java |   3 +-
 .../calcite/rel/convert/ConverterImpl.java      |   7 +-
 .../metadata/CachingRelMetadataProvider.java    |   4 +-
 .../metadata/ChainedRelMetadataProvider.java    |   6 +-
 .../rel/metadata/JaninoRelMetadataProvider.java |  11 +-
 .../rel/metadata/MetadataFactoryImpl.java       |   7 +-
 .../metadata/ReflectiveRelMetadataProvider.java |   5 +-
 .../calcite/rel/rules/CalcRelSplitter.java      |   8 +-
 .../calcite/rel/rules/JoinCommuteRule.java      |   8 +-
 .../calcite/rel/rules/ProjectToWindowRule.java  |   4 +-
 .../calcite/rel/rules/ReduceDecimalsRule.java   |   3 +-
 .../java/org/apache/calcite/rex/RexBuilder.java |   8 +-
 .../java/org/apache/calcite/rex/RexLiteral.java |  11 +-
 .../apache/calcite/rex/RexProgramBuilder.java   |  20 ++--
 .../apache/calcite/schema/impl/ViewTable.java   |   7 +-
 .../org/apache/calcite/sql/SqlCallBinding.java  |   3 +-
 .../java/org/apache/calcite/sql/SqlLiteral.java |   5 +-
 .../java/org/apache/calcite/sql/SqlNode.java    |  16 +--
 .../apache/calcite/sql/SqlPostfixOperator.java  |   3 +-
 .../apache/calcite/sql/SqlPrefixOperator.java   |   3 +-
 .../java/org/apache/calcite/sql/SqlSyntax.java  |   4 +-
 .../java/org/apache/calcite/sql/SqlWindow.java  |   2 +-
 .../apache/calcite/sql/fun/SqlLikeOperator.java |   5 +-
 .../calcite/sql/fun/SqlMinMaxAggFunction.java   |   5 +-
 .../sql/parser/SqlAbstractParserImpl.java       |  12 +--
 .../calcite/sql/parser/SqlParserUtil.java       |  19 ++--
 .../calcite/sql/pretty/SqlPrettyWriter.java     |  14 +--
 .../apache/calcite/sql/type/BasicSqlType.java   |   5 +-
 .../apache/calcite/sql/type/ReturnTypes.java    |   9 +-
 .../sql/type/SqlTypeAssignmentRules.java        |   9 +-
 .../apache/calcite/sql/type/SqlTypeUtil.java    |  12 +--
 .../sql/util/ReflectiveSqlOperatorTable.java    |  10 +-
 .../calcite/sql/validate/AbstractNamespace.java |  11 +-
 .../calcite/sql/validate/SetopNamespace.java    |   3 +-
 .../sql/validate/SqlIdentifierMoniker.java      |   6 +-
 .../calcite/sql/validate/SqlValidatorImpl.java  |  25 +++--
 .../sql2rel/ReflectiveConvertletTable.java      |  57 ++++-------
 .../apache/calcite/sql2rel/RelFieldTrimmer.java |   2 +-
 .../sql2rel/SqlNodeToRexConverterImpl.java      |   5 +-
 .../calcite/sql2rel/SqlToRelConverter.java      |  22 ++--
 .../org/apache/calcite/tools/RelRunners.java    |   4 +-
 .../java/org/apache/calcite/util/Closer.java    |   6 +-
 .../org/apache/calcite/util/ReflectUtil.java    |  16 +--
 .../apache/calcite/util/SaffronProperties.java  |   2 +-
 .../main/java/org/apache/calcite/util/Util.java | 100 ++++++++-----------
 .../calcite/util/javac/JaninoCompiler.java      |  11 +-
 .../plan/volcano/VolcanoPlannerTest.java        |  24 +----
 .../rel/rel2sql/RelToSqlConverterTest.java      |   9 +-
 .../calcite/sql/parser/SqlParserTest.java       |   3 +-
 .../calcite/sql/test/SqlOperatorBaseTest.java   |   9 +-
 .../org/apache/calcite/test/DiffRepository.java |  20 ++--
 .../apache/calcite/test/JdbcAdapterTest.java    |  13 ++-
 .../calcite/test/JdbcFrontLinqBackTest.java     |   3 +-
 .../java/org/apache/calcite/test/JdbcTest.java  |   5 +-
 .../org/apache/calcite/test/QuidemTest.java     |   7 +-
 .../calcite/test/ReflectiveSchemaTest.java      |   5 +-
 .../apache/calcite/test/RelMetadataTest.java    |   3 +-
 .../apache/calcite/test/RelOptRulesTest.java    |  11 +-
 .../org/apache/calcite/test/RexProgramTest.java |   2 +-
 .../org/apache/calcite/test/SqlTestGen.java     |  35 ++-----
 .../apache/calcite/test/SqlToRelTestBase.java   |   8 +-
 .../org/apache/calcite/test/StreamTest.java     |   3 +-
 .../adapter/druid/DruidConnectionImpl.java      |  19 ++--
 .../calcite/adapter/druid/DruidQuery.java       |   3 +-
 .../org/apache/calcite/test/DruidAdapterIT.java |   9 +-
 .../elasticsearch/ElasticsearchSchema.java      |   5 +-
 .../calcite/adapter/csv/CsvEnumerator.java      |   4 +-
 .../calcite/adapter/csv/CsvStreamReader.java    |   4 +-
 .../java/org/apache/calcite/test/CsvTest.java   |   3 +-
 .../calcite/adapter/mongodb/MongoRules.java     |   3 +-
 .../org/apache/calcite/test/MongoAdapterIT.java |   3 +-
 .../org/apache/calcite/test/CalciteHandler.java |   4 +-
 83 files changed, 333 insertions(+), 504 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
----------------------------------------------------------------------
diff --git a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
index ec29bd6..3a9e9e9 100644
--- a/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
+++ b/cassandra/src/main/java/org/apache/calcite/adapter/cassandra/CassandraRules.java
@@ -45,7 +45,6 @@ import org.apache.calcite.util.Pair;
 import com.google.common.base.Predicate;
 import com.google.common.base.Predicates;
 
-import java.util.AbstractList;
 import java.util.HashSet;
 import java.util.List;
 import java.util.Set;
@@ -66,16 +65,8 @@ public class CassandraRules {
   };
 
   static List<String> cassandraFieldNames(final RelDataType rowType) {
-    return SqlValidatorUtil.uniquify(
-        new AbstractList<String>() {
-          @Override public String get(int index) {
-            return rowType.getFieldList().get(index).getName();
-          }
-
-          @Override public int size() {
-            return rowType.getFieldCount();
-          }
-        });
+    return SqlValidatorUtil.uniquify(rowType.getFieldNames(),
+        SqlValidatorUtil.EXPR_SUGGESTER, true);
   }
 
   /** Translator from {@link RexNode} to strings in Cassandra's expression

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 931f38f..979a707 100644
--- a/core/src/main/codegen/templates/Parser.jj
+++ b/core/src/main/codegen/templates/Parser.jj
@@ -100,6 +100,7 @@ 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.Glossary;
+import org.apache.calcite.util.NlsString;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
@@ -1883,7 +1884,8 @@ SqlNode TableRef2(boolean lateral) :
         (
             <SUBSTITUTE> <LPAREN> sample = StringLiteral() <RPAREN>
             {
-                String sampleName = SqlLiteral.stringValue(sample);
+                String sampleName =
+                    ((NlsString) SqlLiteral.value(sample)).getValue();
                 SqlSampleSpec sampleSpec = SqlSampleSpec.createNamed(sampleName);
                 SqlLiteral sampleLiteral = SqlLiteral.createSample(sampleSpec, pos);
                 tableRef = SqlStdOperatorTable.TABLESAMPLE.createCall(

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java b/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
index 1cffac4..a342b25 100644
--- a/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
+++ b/core/src/main/java/org/apache/calcite/interpreter/AggregateNode.java
@@ -41,7 +41,6 @@ import org.apache.calcite.util.ImmutableBitSet;
 import org.apache.calcite.util.Pair;
 
 import com.google.common.base.Supplier;
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -451,7 +450,7 @@ public class AggregateNode extends AbstractSingleNode<Aggregate> {
         try {
           instance = aggFunction.declaringClass.newInstance();
         } catch (InstantiationException | IllegalAccessException e) {
-          throw Throwables.propagate(e);
+          throw new RuntimeException(e);
         }
       }
     }
@@ -471,7 +470,7 @@ public class AggregateNode extends AbstractSingleNode<Aggregate> {
       try {
         this.value = factory.aggFunction.initMethod.invoke(factory.instance);
       } catch (IllegalAccessException | InvocationTargetException e) {
-        throw Throwables.propagate(e);
+        throw new RuntimeException(e);
       }
     }
 
@@ -485,7 +484,7 @@ public class AggregateNode extends AbstractSingleNode<Aggregate> {
       try {
         value = factory.aggFunction.addMethod.invoke(factory.instance, args);
       } catch (IllegalAccessException | InvocationTargetException e) {
-        throw Throwables.propagate(e);
+        throw new RuntimeException(e);
       }
     }
 
@@ -494,7 +493,7 @@ public class AggregateNode extends AbstractSingleNode<Aggregate> {
       try {
         return factory.aggFunction.resultMethod.invoke(factory.instance, args);
       } catch (IllegalAccessException | InvocationTargetException e) {
-        throw Throwables.propagate(e);
+        throw new RuntimeException(e);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 98a1994..5717603 100644
--- a/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
+++ b/core/src/main/java/org/apache/calcite/jdbc/CalciteConnectionImpl.java
@@ -61,7 +61,6 @@ import org.apache.calcite.util.BuiltInMethod;
 import org.apache.calcite.util.Holder;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.Maps;
@@ -163,7 +162,7 @@ abstract class CalciteConnectionImpl
                     ResultSet.TYPE_FORWARD_ONLY, ResultSet.CONCUR_READ_ONLY,
                     getHoldability());
               } catch (SQLException e) {
-                throw Throwables.propagate(e);
+                throw new RuntimeException(e);
               }
             }
           });
@@ -288,7 +287,7 @@ abstract class CalciteConnectionImpl
     try {
       cancelFlag = getCancelFlag(handle);
     } catch (NoSuchStatementException e) {
-      throw Throwables.propagate(e);
+      throw new RuntimeException(e);
     }
     map.put(DataContext.Variable.CANCEL_FLAG.camelName, cancelFlag);
     final DataContext dataContext = createDataContext(map);

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/materialize/CachingLatticeStatisticProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/materialize/CachingLatticeStatisticProvider.java b/core/src/main/java/org/apache/calcite/materialize/CachingLatticeStatisticProvider.java
index bf2702c..5ab6cf6 100644
--- a/core/src/main/java/org/apache/calcite/materialize/CachingLatticeStatisticProvider.java
+++ b/core/src/main/java/org/apache/calcite/materialize/CachingLatticeStatisticProvider.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.materialize;
 
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import com.google.common.base.Throwables;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -50,7 +50,8 @@ class CachingLatticeStatisticProvider implements LatticeStatisticProvider {
     try {
       return cache.get(Pair.of(lattice, column));
     } catch (UncheckedExecutionException | ExecutionException e) {
-      throw Throwables.propagate(e.getCause());
+      Util.throwIfUnchecked(e.getCause());
+      throw new RuntimeException(e.getCause());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java b/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
index eddbb89..a46ee73 100644
--- a/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
+++ b/core/src/main/java/org/apache/calcite/plan/ConventionTraitDef.java
@@ -20,7 +20,6 @@ import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.convert.ConverterRule;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
 import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
 import org.apache.calcite.util.graph.DirectedGraph;
@@ -159,8 +158,7 @@ public class ConventionTraitDef extends RelTraitDef<Convention> {
                   converted, previous, arc,
                   conversionData.mapArcToConverterRule);
           if (converted == null) {
-            throw Util.newInternal("Converter from " + previous
-                + " to " + arc
+            throw new AssertionError("Converter from " + previous + " to " + arc
                 + " guaranteed that it could convert any relexp");
           }
         }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 3239419..37bb54a 100644
--- a/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
+++ b/core/src/main/java/org/apache/calcite/plan/RelOptUtil.java
@@ -308,8 +308,8 @@ public abstract class RelOptUtil {
       RelNode p) {
     try {
       visitor.go(p);
-    } catch (Throwable e) {
-      throw Util.newInternal(e, "while visiting tree");
+    } catch (Exception e) {
+      throw new RuntimeException("while visiting tree", e);
     }
   }
 
@@ -382,7 +382,7 @@ public abstract class RelOptUtil {
         + "\nexpression type is " + actualRowType.getFullTypeString()
         + "\nset is " + equivalenceClass.toString()
         + "\nexpression is " + newRel.toString();
-    throw Util.newInternal(s);
+    throw new AssertionError(s);
   }
 
   /**
@@ -1128,10 +1128,9 @@ public abstract class RelOptUtil {
                     ImmutableList.of(leftKeyType, rightKeyType));
 
             if (targetKeyType == null) {
-              throw Util.newInternal(
-                  "Cannot find common type for join keys "
-                  + leftKey + " (type " + leftKeyType + ") and "
-                  + rightKey + " (type " + rightKeyType + ")");
+              throw new AssertionError("Cannot find common type for join keys "
+                  + leftKey + " (type " + leftKeyType + ") and " + rightKey
+                  + " (type " + rightKeyType + ")");
             }
 
             if (leftKeyType != targetKeyType) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java b/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
index e029872..1473f8f 100644
--- a/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
+++ b/core/src/main/java/org/apache/calcite/plan/hep/HepPlanner.java
@@ -924,9 +924,8 @@ public class HepPlanner extends AbstractRelOptPlanner {
       return;
     }
 
-    throw Util.newInternal(
-        "Query graph cycle detected in HepPlanner:  "
-            + cyclicVertices);
+    throw new AssertionError("Query graph cycle detected in HepPlanner: "
+        + cyclicVertices);
   }
 
   private void dumpGraph() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 39b2857..15dbb98 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
@@ -1093,9 +1093,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
       try {
         schema.registerRules(this);
       } catch (Exception e) {
-        throw Util.newInternal(
-            e,
-            "Error while registering schema " + schema);
+        throw new AssertionError("While registering schema " + schema, e);
       }
     }
   }
@@ -1600,7 +1598,7 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     if (p != null) {
       p = p.equivalentSet;
       if (p == s) {
-        throw Util.newInternal("cycle in equivalence tree");
+        throw new AssertionError("cycle in equivalence tree");
       }
     }
     return p;
@@ -1626,9 +1624,8 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
 
     assert !isRegistered(rel) : "already been registered: " + rel;
     if (rel.getCluster().getPlanner() != this) {
-      throw Util.newInternal("Relational expression " + rel
-          + " belongs to a different planner than is currently being"
-          + " used.");
+      throw new AssertionError("Relational expression " + rel
+          + " belongs to a different planner than is currently being used.");
     }
 
     // Now is a good time to ensure that the relational expression
@@ -1638,17 +1635,15 @@ public class VolcanoPlanner extends AbstractRelOptPlanner {
     assert convention != null;
     if (!convention.getInterface().isInstance(rel)
         && !(rel instanceof Converter)) {
-      throw Util.newInternal(
-          "Relational expression " + rel
-              + " has calling-convention " + convention
-              + " but does not implement the required interface '"
-              + convention.getInterface() + "' of that convention");
+      throw new AssertionError("Relational expression " + rel
+          + " has calling-convention " + convention
+          + " but does not implement the required interface '"
+          + convention.getInterface() + "' of that convention");
     }
     if (traits.size() != traitDefs.size()) {
-      throw Util.newInternal(
-          "Relational expression " + rel
-          + " does not have the correct number of traits "
-          + traits.size() + " != " + traitDefs.size());
+      throw new AssertionError("Relational expression " + rel
+          + " does not have the correct number of traits: " + traits.size()
+          + " != " + traitDefs.size());
     }
 
     // Ensure that its sub-expressions are registered.

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
index 8372821..7a629fe 100644
--- a/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
+++ b/core/src/main/java/org/apache/calcite/plan/volcano/VolcanoRuleCall.java
@@ -22,7 +22,6 @@ import org.apache.calcite.plan.RelOptRuleOperand;
 import org.apache.calcite.plan.RelTraitPropagationVisitor;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
-import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -144,10 +143,9 @@ public class VolcanoRuleCall extends RelOptRuleCall {
                 false);
         volcanoPlanner.listener.ruleProductionSucceeded(event);
       }
-    } catch (Throwable e) {
-      throw Util.newInternal(
-          e,
-          "Error occurred while applying rule " + getRule());
+    } catch (Exception e) {
+      throw new RuntimeException("Error occurred while applying rule "
+          + getRule(), e);
     }
   }
 
@@ -232,10 +230,9 @@ public class VolcanoRuleCall extends RelOptRuleCall {
                 false);
         volcanoPlanner.listener.ruleAttempted(event);
       }
-    } catch (Throwable e) {
-      throw Util.newInternal(e,
-          "Error while applying rule "
-          + getRule() + ", args " + Arrays.toString(rels));
+    } catch (Exception e) {
+      throw new RuntimeException("Error while applying rule " + getRule()
+          + ", args " + Arrays.toString(rels), e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 5592a76..78ac0da 100644
--- a/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
+++ b/core/src/main/java/org/apache/calcite/rel/AbstractRelNode.java
@@ -361,7 +361,7 @@ public abstract class AbstractRelNode implements RelNode {
   public void replaceInput(
       int ordinalInParent,
       RelNode p) {
-    throw Util.newInternal("replaceInput called on " + this);
+    throw new UnsupportedOperationException("replaceInput called on " + this);
   }
 
   public String toString() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/BiRel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/BiRel.java b/core/src/main/java/org/apache/calcite/rel/BiRel.java
index c72f5ea..6e3a4f2 100644
--- a/core/src/main/java/org/apache/calcite/rel/BiRel.java
+++ b/core/src/main/java/org/apache/calcite/rel/BiRel.java
@@ -19,7 +19,6 @@ package org.apache.calcite.rel;
 import org.apache.calcite.plan.RelOptCluster;
 import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.runtime.FlatLists;
-import org.apache.calcite.util.Util;
 
 import java.util.List;
 
@@ -70,7 +69,7 @@ public abstract class BiRel extends AbstractRelNode {
       this.right = p;
       break;
     default:
-      throw Util.newInternal();
+      throw new IndexOutOfBoundsException("Input " + ordinalInParent);
     }
     recomputeDigest();
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java b/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
index b0cb40b..acadd74 100644
--- a/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/convert/ConverterImpl.java
@@ -24,7 +24,6 @@ import org.apache.calcite.plan.RelTraitSet;
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.rel.SingleRel;
 import org.apache.calcite.rel.metadata.RelMetadataQuery;
-import org.apache.calcite.util.Util;
 
 /**
  * Abstract implementation of {@link Converter}.
@@ -66,10 +65,10 @@ public abstract class ConverterImpl extends SingleRel
     return planner.getCostFactory().makeCost(dRows, dCpu, dIo);
   }
 
+  @Deprecated // to be removed before 2.0
   protected Error cannotImplement() {
-    return Util.newInternal(
-        getClass() + " cannot convert from "
-            + inTraits + " traits");
+    return new AssertionError(getClass() + " cannot convert from "
+        + inTraits + " traits");
   }
 
   public RelTraitSet getInputTraits() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
index eb63604..478190c 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/CachingRelMetadataProvider.java
@@ -20,7 +20,6 @@ import org.apache.calcite.plan.RelOptPlanner;
 import org.apache.calcite.rel.RelNode;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Multimap;
 
@@ -141,8 +140,7 @@ public class CachingRelMetadataProvider implements RelMetadataProvider {
         }
         return result;
       } catch (InvocationTargetException e) {
-        Throwables.propagateIfPossible(e.getCause());
-        throw e;
+        throw e.getCause();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
index 1ecec0f..8643f91 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ChainedRelMetadataProvider.java
@@ -17,8 +17,8 @@
 package org.apache.calcite.rel.metadata;
 
 import org.apache.calcite.rel.RelNode;
+import org.apache.calcite.util.Util;
 
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Lists;
@@ -140,8 +140,8 @@ public class ChainedRelMetadataProvider implements RelMetadataProvider {
           if (e.getCause() instanceof CyclicMetadataException) {
             continue;
           }
-          Throwables.propagateIfPossible(e.getCause());
-          throw e;
+          Util.throwIfUnchecked(e.getCause());
+          throw new RuntimeException(e.getCause());
         }
       }
       return null;

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
index 9e32159..42de769 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/JaninoRelMetadataProvider.java
@@ -54,8 +54,8 @@ import org.apache.calcite.rel.stream.LogicalDelta;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.util.ControlFlowException;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import com.google.common.base.Throwables;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -367,8 +367,8 @@ public class JaninoRelMetadataProvider implements RelMetadataProvider {
     try {
       return compile(decl, buff.toString(), def, argList);
     } catch (CompileException | IOException e) {
-      System.out.println(buff);
-      throw Throwables.propagate(e);
+      throw new RuntimeException("Error compiling:\n"
+          + buff, e);
     }
   }
 
@@ -444,7 +444,7 @@ public class JaninoRelMetadataProvider implements RelMetadataProvider {
     } catch (InstantiationException
         | IllegalAccessException
         | InvocationTargetException e) {
-      throw Throwables.propagate(e);
+      throw new RuntimeException(e);
     }
     return def.handlerClass.cast(o);
   }
@@ -457,7 +457,8 @@ public class JaninoRelMetadataProvider implements RelMetadataProvider {
       //noinspection unchecked
       return (H) HANDLERS.get(key);
     } catch (UncheckedExecutionException | ExecutionException e) {
-      throw Throwables.propagate(e.getCause());
+      Util.throwIfUnchecked(e.getCause());
+      throw new RuntimeException(e.getCause());
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
index 09446be..063cf6e 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/MetadataFactoryImpl.java
@@ -18,8 +18,8 @@ package org.apache.calcite.rel.metadata;
 
 import org.apache.calcite.rel.RelNode;
 import org.apache.calcite.util.Pair;
+import org.apache.calcite.util.Util;
 
-import com.google.common.base.Throwables;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
@@ -50,7 +50,7 @@ public class MetadataFactoryImpl implements MetadataFactory {
     this.cache = CacheBuilder.newBuilder().build(loader(provider));
   }
 
-  static CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
+  private static CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
       UnboundMetadata<Metadata>> loader(final RelMetadataProvider provider) {
     return new CacheLoader<Pair<Class<RelNode>, Class<Metadata>>,
         UnboundMetadata<Metadata>>() {
@@ -73,7 +73,8 @@ public class MetadataFactoryImpl implements MetadataFactory {
       final Metadata apply = cache.get(key).bind(rel, mq);
       return metadataClazz.cast(apply);
     } catch (UncheckedExecutionException | ExecutionException e) {
-      throw Throwables.propagate(e.getCause());
+      Util.throwIfUnchecked(e.getCause());
+      throw new RuntimeException(e.getCause());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
index baceb35..bdaf4d3 100644
--- a/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
+++ b/core/src/main/java/org/apache/calcite/rel/metadata/ReflectiveRelMetadataProvider.java
@@ -26,7 +26,6 @@ import org.apache.calcite.util.ReflectiveVisitor;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
-import com.google.common.base.Throwables;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMultimap;
 import com.google.common.collect.Multimap;
@@ -197,8 +196,8 @@ public class ReflectiveRelMetadataProvider
                         return handlerMethod.invoke(target, args1);
                       } catch (InvocationTargetException
                           | UndeclaredThrowableException e) {
-                        Throwables.propagateIfPossible(e.getCause());
-                        throw e;
+                        Util.throwIfUnchecked(e.getCause());
+                        throw new RuntimeException(e.getCause());
                       } finally {
                         mq.map.remove(key);
                       }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 a29dd97..5a19e56 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
@@ -43,6 +43,7 @@ import org.apache.calcite.util.graph.DefaultEdge;
 import org.apache.calcite.util.graph.DirectedGraph;
 import org.apache.calcite.util.graph.TopologicalOrderIterator;
 
+import com.google.common.base.Preconditions;
 import com.google.common.primitives.Ints;
 
 import org.slf4j.Logger;
@@ -240,8 +241,7 @@ public abstract class CalcRelSplitter {
       inputExprOrdinals = projectExprOrdinals;
     }
 
-    Util.permAssert(
-        doneCondition || (conditionRef == null),
+    Preconditions.checkArgument(doneCondition || (conditionRef == null),
         "unhandled condition");
     return rel;
   }
@@ -371,7 +371,7 @@ public abstract class CalcRelSplitter {
           // level, with all options open?
           if (count(relTypesPossibleForTopLevel) >= relTypes.length) {
             // Cannot implement for any type.
-            throw Util.newInternal("cannot implement " + expr);
+            throw new AssertionError("cannot implement " + expr);
           }
           levelTypeOrdinals[levelCount] =
               firstSet(relTypesPossibleForTopLevel);
@@ -712,7 +712,7 @@ public abstract class CalcRelSplitter {
         return relType.canImplement(rel.getProgram());
       }
     }
-    throw Util.newInternal("unknown type " + relTypeName);
+    throw new AssertionError("unknown type " + relTypeName);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java b/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
index 863bf25..28d48af 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/JoinCommuteRule.java
@@ -34,7 +34,6 @@ import org.apache.calcite.rex.RexInputRef;
 import org.apache.calcite.rex.RexNode;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.tools.RelBuilderFactory;
-import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
@@ -100,6 +99,10 @@ public class JoinCommuteRule extends RelOptRule {
   }
 
   /**
+   * Returns a relational expression with the inputs switched round. Does not
+   * modify <code>join</code>. Returns null if the join cannot be swapped (for
+   * example, because it is an outer join).
+   *
    * @param join           join to be swapped
    * @param swapOuterJoins whether outer joins should be swapped
    * @return swapped join if swapping possible; else null
@@ -220,8 +223,7 @@ public class JoinCommuteRule extends RelOptRule {
               rightFields.get(index).getType(),
               index);
         }
-        throw Util.newInternal("Bad field offset: index="
-            + var.getIndex()
+        throw new AssertionError("Bad field offset: index=" + var.getIndex()
             + ", leftFieldCount=" + leftFields.size()
             + ", rightFieldCount=" + rightFields.size());
       } else {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
index a2856f5..054f4c6 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ProjectToWindowRule.java
@@ -41,13 +41,13 @@ import org.apache.calcite.runtime.PredicateImpl;
 import org.apache.calcite.tools.RelBuilder;
 import org.apache.calcite.util.ImmutableIntList;
 import org.apache.calcite.util.Pair;
-import org.apache.calcite.util.Util;
 import org.apache.calcite.util.graph.DefaultDirectedGraph;
 import org.apache.calcite.util.graph.DefaultEdge;
 import org.apache.calcite.util.graph.DirectedGraph;
 import org.apache.calcite.util.graph.TopologicalOrderIterator;
 
 import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Sets;
@@ -226,7 +226,7 @@ public abstract class ProjectToWindowRule extends RelOptRule {
 
         protected RelNode makeRel(RelOptCluster cluster, RelTraitSet traitSet,
             RelBuilder relBuilder, RelNode input, RexProgram program) {
-          Util.permAssert(program.getCondition() == null,
+          Preconditions.checkArgument(program.getCondition() == null,
               "WindowedAggregateRel cannot accept a condition");
           return LogicalWindow.create(cluster, traitSet, relBuilder, input,
               program);

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java b/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
index ef0af42..5154a15 100644
--- a/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
+++ b/core/src/main/java/org/apache/calcite/rel/rules/ReduceDecimalsRule.java
@@ -868,8 +868,7 @@ public class ReduceDecimalsRule extends RelOptRule {
       } else if (call.getOperator() == SqlStdOperatorTable.MOD) {
         return expandMod(call, operands);
       } else {
-        throw Util.newInternal(
-            "ReduceDecimalsRule could not expand "
+        throw new AssertionError("ReduceDecimalsRule could not expand "
             + call.getOperator());
       }
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 87c84e3..d8edee9 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexBuilder.java
@@ -48,6 +48,7 @@ import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Lists;
 
@@ -168,8 +169,7 @@ public class RexBuilder {
     final RelDataTypeField field =
         type.getField(fieldName, caseSensitive, false);
     if (field == null) {
-      throw Util.newInternal(
-          "Type '" + type + "' has no field '"
+      throw new AssertionError("Type '" + type + "' has no field '"
           + fieldName + "'");
     }
     return makeFieldAccessInternal(expr, field);
@@ -189,7 +189,7 @@ public class RexBuilder {
     final RelDataType type = expr.getType();
     final List<RelDataTypeField> fields = type.getFieldList();
     if ((i < 0) || (i >= fields.size())) {
-      throw Util.newInternal("Field ordinal " + i + " is invalid for "
+      throw new AssertionError("Field ordinal " + i + " is invalid for "
           + " type '" + type + "'");
     }
     return makeFieldAccessInternal(expr, fields.get(i));
@@ -374,7 +374,7 @@ public class RexBuilder {
           makeCast(type, constantNull()));
     }
     if (!allowPartial) {
-      Util.permAssert(physical, "DISALLOW PARTIAL over RANGE");
+      Preconditions.checkArgument(physical, "DISALLOW PARTIAL over RANGE");
       final RelDataType bigintType = getTypeFactory().createSqlType(
           SqlTypeName.BIGINT);
       // todo: read bound

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
index c7c9b56..1d1e14d 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexLiteral.java
@@ -532,16 +532,15 @@ public class RexLiteral extends RexNode {
         }
       }
       if (cal == null) {
-        throw Util.newInternal(
-            "fromJdbcString: invalid date/time value '"
-                + literal + "'");
+        throw new AssertionError("fromJdbcString: invalid date/time value '"
+            + literal + "'");
       }
       return new RexLiteral(cal, type, typeName);
     case SYMBOL:
 
       // Symbols are for internal use
     default:
-      throw Util.newInternal("fromJdbcString: unsupported type");
+      throw new AssertionError("fromJdbcString: unsupported type");
     }
   }
 
@@ -554,7 +553,7 @@ public class RexLiteral extends RexNode {
     case TIMESTAMP:
       return DateTimeUtils.TIMESTAMP_FORMAT_STRING;
     default:
-      throw Util.newInternal("getCalendarFormat: unknown type");
+      throw new AssertionError("getCalendarFormat: unknown type");
     }
   }
 
@@ -676,7 +675,7 @@ public class RexLiteral extends RexNode {
         return value.negate();
       }
     }
-    throw Util.newInternal("not a literal: " + node);
+    throw new AssertionError("not a literal: " + node);
   }
 
   public static boolean isNullLiteral(RexNode node) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 3a20448..2fadc90 100644
--- a/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
+++ b/core/src/main/java/org/apache/calcite/rex/RexProgramBuilder.java
@@ -22,7 +22,6 @@ 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;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -163,21 +162,18 @@ public class RexProgramBuilder {
             if (index < fields.size()) {
               final RelDataTypeField inputField = fields.get(index);
               if (input.getType() != inputField.getType()) {
-                throw Util.newInternal("in expression " + expr
-                    + ", field reference " + input
-                    + " has inconsistent type");
+                throw new AssertionError("in expression " + expr
+                    + ", field reference " + input + " has inconsistent type");
               }
             } else {
               if (index >= fieldOrdinal) {
-                throw Util.newInternal("in expression " + expr
-                    + ", field reference " + input
-                    + " is out of bounds");
+                throw new AssertionError("in expression " + expr
+                    + ", field reference " + input + " is out of bounds");
               }
               RexNode refExpr = exprList.get(index);
               if (refExpr.getType() != input.getType()) {
-                throw Util.newInternal("in expression " + expr
-                    + ", field reference " + input
-                    + " has inconsistent type");
+                throw new AssertionError("in expression " + expr
+                    + ", field reference " + input + " has inconsistent type");
               }
             }
             return null;
@@ -951,8 +947,8 @@ public class RexProgramBuilder {
         if (expr instanceof RexLocalRef) {
           local = (RexLocalRef) expr;
           if (local.index >= index) {
-            throw Util.newInternal("expr " + local
-                + " references later expr " + local.index);
+            throw new AssertionError(
+                "expr " + local + " references later expr " + local.index);
           }
         } else {
           // Add expression to the list, just so that subsequent

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java b/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
index 0a5d006..507db2f 100644
--- a/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
+++ b/core/src/main/java/org/apache/calcite/schema/impl/ViewTable.java
@@ -42,7 +42,6 @@ import org.apache.calcite.schema.Table;
 import org.apache.calcite.schema.TableMacro;
 import org.apache.calcite.schema.TranslatableTable;
 import org.apache.calcite.util.ImmutableIntList;
-import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 
@@ -134,9 +133,9 @@ public class ViewTable
 
       root = root.withRel(RelOptUtil.createCastRel(root.rel, rowType, true));
       return root;
-    } catch (Throwable e) {
-      throw Util.newInternal(
-          e, "Error while parsing view definition:  " + queryString);
+    } catch (Exception e) {
+      throw new RuntimeException("Error while parsing view definition: "
+          + queryString, e);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
index ae8abae..77e4b52 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlCallBinding.java
@@ -29,7 +29,6 @@ import org.apache.calcite.sql.validate.SqlValidatorNamespace;
 import org.apache.calcite.sql.validate.SqlValidatorScope;
 import org.apache.calcite.sql.validate.SqlValidatorUtil;
 import org.apache.calcite.util.NlsString;
-import org.apache.calcite.util.Util;
 
 import com.google.common.base.Function;
 import com.google.common.collect.Lists;
@@ -211,7 +210,7 @@ public class SqlCallBinding extends SqlOperatorBinding {
             RESOURCE.numberLiteralOutOfRange(bd.toString()));
       }
     }
-    throw Util.newInternal("should never come here");
+    throw new AssertionError();
   }
 
   @Override public Comparable getOperandLiteralValue(int ordinal) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 3f29975..228abfc 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlLiteral.java
@@ -371,9 +371,10 @@ public class SqlLiteral extends SqlNode {
       return literal.toValue();
     } else if (node instanceof SqlCall
         && ((SqlCall) node).getOperator() == SqlStdOperatorTable.CAST) {
+      //noinspection deprecation
       return stringValue(((SqlCall) node).operand(0));
     } else {
-      throw Util.newInternal("invalid string literal: " + node);
+      throw new AssertionError("invalid string literal: " + node);
     }
   }
 
@@ -387,7 +388,7 @@ public class SqlLiteral extends SqlNode {
     } else if (SqlUtil.isLiteralChain(node)) {
       return SqlLiteralChainOperator.concatenateOperands((SqlCall) node);
     } else {
-      throw Util.newInternal("invalid literal: " + node);
+      throw new AssertionError("invalid literal: " + node);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 b31cf7b..e7c77fb 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlNode.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlNode.java
@@ -27,6 +27,8 @@ 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;
+
 import java.util.Collection;
 import java.util.List;
 import java.util.Set;
@@ -55,8 +57,7 @@ public abstract class SqlNode implements Cloneable {
    * @param pos Parser position, must not be null.
    */
   SqlNode(SqlParserPos pos) {
-    Util.pre(pos != null, "pos != null");
-    this.pos = pos;
+    this.pos = Preconditions.checkNotNull(pos);
   }
 
   //~ Methods ----------------------------------------------------------------
@@ -68,16 +69,7 @@ public abstract class SqlNode implements Cloneable {
   /**
    * Clones a SqlNode with a different position.
    */
-  public SqlNode clone(SqlParserPos pos) {
-    // REVIEW jvs 26-July-2006:  shouldn't pos be used here?  Or are
-    // subclasses always supposed to override, in which case this
-    // method should probably be abstract?
-    try {
-      return (SqlNode) super.clone();
-    } catch (CloneNotSupportedException e) {
-      throw Util.newInternal(e, "error while cloning " + this);
-    }
-  }
+  public abstract SqlNode clone(SqlParserPos pos);
 
   /**
    * Returns the type of node this is, or

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 31020d1..4c69252 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPostfixOperator.java
@@ -69,8 +69,7 @@ public class SqlPostfixOperator extends SqlOperator {
       RelDataType operandType =
           validator.getValidatedNodeType(call.operand(0));
       if (null == operandType) {
-        throw Util.newInternal(
-            "operand's type should have been derived");
+        throw new AssertionError("operand's type should have been derived");
       }
       if (SqlTypeUtil.inCharFamily(operandType)) {
         SqlCollation collation = operandType.getCollation();

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 af44dd2..a28e727 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlPrefixOperator.java
@@ -70,8 +70,7 @@ public class SqlPrefixOperator extends SqlOperator {
       RelDataType operandType =
           validator.getValidatedNodeType(call.operand(0));
       if (null == operandType) {
-        throw Util.newInternal(
-            "operand's type should have been derived");
+        throw new AssertionError("operand's type should have been derived");
       }
       if (SqlTypeUtil.inCharFamily(operandType)) {
         SqlCollation collation = operandType.getCollation();

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java b/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java
index f3e9bbc..50d30ac 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlSyntax.java
@@ -141,8 +141,8 @@ public enum SqlSyntax {
         SqlCall call,
         int leftPrec,
         int rightPrec) {
-      throw Util.newInternal("Internal operator '" + operator
-          + "' cannot be un-parsed");
+      throw new UnsupportedOperationException("Internal operator '"
+          + operator + "' " + "cannot be un-parsed");
     }
   };
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
index 86adcc6..d280ac2 100644
--- a/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
+++ b/core/src/main/java/org/apache/calcite/sql/SqlWindow.java
@@ -698,7 +698,7 @@ public class SqlWindow extends SqlCall {
       }
       break;
     default:
-      throw Util.newInternal("Unexpected node type");
+      throw new AssertionError("Unexpected node type");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/fun/SqlLikeOperator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlLikeOperator.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlLikeOperator.java
index 9a0187c..c53740f 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlLikeOperator.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlLikeOperator.java
@@ -31,7 +31,6 @@ import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
 import org.apache.calcite.sql.type.SqlOperandCountRanges;
 import org.apache.calcite.sql.type.SqlTypeUtil;
-import org.apache.calcite.util.Util;
 
 /**
  * An operator describing the <code>LIKE</code> and <code>SIMILAR</code>
@@ -119,8 +118,8 @@ public class SqlLikeOperator extends SqlSpecialOperator {
       // enforce the escape character length to be 1
       break;
     default:
-      throw Util.newInternal(
-          "unexpected number of args to " + callBinding.getCall());
+      throw new AssertionError("unexpected number of args to "
+          + callBinding.getCall() + ": " + callBinding.getOperandCount());
     }
 
     return SqlTypeUtil.isCharTypeComparable(

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
index 391c09a..54af0f8 100644
--- a/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
+++ b/core/src/main/java/org/apache/calcite/sql/fun/SqlMinMaxAggFunction.java
@@ -24,7 +24,6 @@ import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlSplittableAggFunction;
 import org.apache.calcite.sql.type.OperandTypes;
 import org.apache.calcite.sql.type.ReturnTypes;
-import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
@@ -113,7 +112,7 @@ public class SqlMinMaxAggFunction extends SqlAggFunction {
     case MINMAX_COMPARATOR:
       return argTypes.subList(1, 2);
     default:
-      throw Util.newInternal("bad kind: " + minMaxKind);
+      throw new AssertionError("bad kind: " + minMaxKind);
     }
   }
 
@@ -125,7 +124,7 @@ public class SqlMinMaxAggFunction extends SqlAggFunction {
     case MINMAX_COMPARATOR:
       return argTypes.get(1);
     default:
-      throw Util.newInternal("bad kind: " + minMaxKind);
+      throw new AssertionError("bad kind: " + minMaxKind);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/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 6c7b167..f4631f6 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
@@ -28,7 +28,6 @@ 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.Glossary;
-import org.apache.calcite.util.Util;
 
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableSet;
@@ -591,8 +590,7 @@ public abstract class SqlAbstractParserImpl {
       parserImpl.ReInit(new StringReader("1"));
       try {
         Object o = virtualCall(parserImpl, name);
-        Util.discard(o);
-        throw Util.newInternal("expected call to fail");
+        throw new AssertionError("expected call to fail, got " + o);
       } catch (SqlParseException parseException) {
         // First time through, build the list of all tokens.
         final String[] tokenImages = parseException.getTokenImages();
@@ -619,9 +617,7 @@ public abstract class SqlAbstractParserImpl {
           }
         }
       } catch (Throwable e) {
-        throw Util.newInternal(
-            e,
-            "Unexpected error while building token lists");
+        throw new RuntimeException("While building token lists", e);
       }
     }
 
@@ -640,10 +636,6 @@ public abstract class SqlAbstractParserImpl {
       try {
         final Method method = clazz.getMethod(name, (Class[]) null);
         return method.invoke(parserImpl, (Object[]) null);
-      } catch (NoSuchMethodException e) {
-        throw Util.newInternal(e);
-      } catch (IllegalAccessException e) {
-        throw Util.newInternal(e);
       } catch (InvocationTargetException e) {
         Throwable cause = e.getCause();
         throw parserImpl.normalizeException(cause);

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
index c9e1ad7..228b5f7 100644
--- a/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/parser/SqlParserUtil.java
@@ -38,6 +38,7 @@ import org.apache.calcite.util.SaffronProperties;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteTrace;
 
+import com.google.common.base.Preconditions;
 import com.google.common.base.Predicate;
 
 import org.slf4j.Logger;
@@ -155,8 +156,7 @@ public final class SqlParserUtil {
   public static long intervalToMillis(
       String literal,
       SqlIntervalQualifier intervalQualifier) {
-    Util.permAssert(
-        !intervalQualifier.isYearMonth(),
+    Preconditions.checkArgument(!intervalQualifier.isYearMonth(),
         "interval must be day time");
     int[] ret;
     try {
@@ -164,8 +164,8 @@ public final class SqlParserUtil {
           intervalQualifier.getParserPosition(), RelDataTypeSystem.DEFAULT);
       assert ret != null;
     } catch (CalciteContextException e) {
-      throw Util.newInternal(
-          e, "while parsing day-to-second interval " + literal);
+      throw new RuntimeException("while parsing day-to-second interval "
+          + literal, e);
     }
     long l = 0;
     long[] conv = new long[5];
@@ -197,8 +197,7 @@ public final class SqlParserUtil {
   public static long intervalToMonths(
       String literal,
       SqlIntervalQualifier intervalQualifier) {
-    Util.permAssert(
-        intervalQualifier.isYearMonth(),
+    Preconditions.checkArgument(intervalQualifier.isYearMonth(),
         "interval must be year month");
     int[] ret;
     try {
@@ -206,8 +205,8 @@ public final class SqlParserUtil {
           intervalQualifier.getParserPosition(), RelDataTypeSystem.DEFAULT);
       assert ret != null;
     } catch (CalciteContextException e) {
-      throw Util.newInternal(
-          e, "error parsing year-to-month interval " + literal);
+      throw new RuntimeException("Error while parsing year-to-month interval "
+          + literal, e);
     }
 
     long l = 0;
@@ -542,8 +541,8 @@ public final class SqlParserUtil {
       int start,
       int end,
       T o) {
-    Util.pre(list != null, "list != null");
-    Util.pre(start < end, "start < end");
+    Preconditions.checkNotNull(list);
+    Preconditions.checkArgument(start < end);
     for (int i = end - 1; i > start; --i) {
       list.remove(i);
     }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
index e212a5d..c2b206a 100644
--- a/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
+++ b/core/src/main/java/org/apache/calcite/sql/pretty/SqlPrettyWriter.java
@@ -25,7 +25,7 @@ import org.apache.calcite.sql.util.SqlString;
 import org.apache.calcite.util.Util;
 import org.apache.calcite.util.trace.CalciteLogger;
 
-import com.google.common.base.Throwables;
+import com.google.common.base.Preconditions;
 
 import org.slf4j.LoggerFactory;
 
@@ -763,10 +763,8 @@ public class SqlPrettyWriter implements SqlWriter {
 
   public void endList(Frame frame) {
     FrameImpl endedFrame = (FrameImpl) frame;
-    Util.pre(
-        frame == this.frame,
-        "Frame " + endedFrame.frameType
-            + " does not match current frame " + this.frame.frameType);
+    Preconditions.checkArgument(frame == this.frame,
+        "Frame does not match current frame");
     if (this.frame == null) {
       throw new RuntimeException("No list started");
     }
@@ -1159,7 +1157,8 @@ public class SqlPrettyWriter implements SqlWriter {
       try {
         method.invoke(o, value);
       } catch (IllegalAccessException | InvocationTargetException e) {
-        throw Throwables.propagate(e);
+        Util.throwIfUnchecked(e.getCause());
+        throw new RuntimeException(e.getCause());
       }
     }
 
@@ -1168,7 +1167,8 @@ public class SqlPrettyWriter implements SqlWriter {
       try {
         return method.invoke(o);
       } catch (IllegalAccessException | InvocationTargetException e) {
-        throw Throwables.propagate(e);
+        Util.throwIfUnchecked(e.getCause());
+        throw new RuntimeException(e.getCause());
       }
     }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java b/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
index eb699ef..04d1a88 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/BasicSqlType.java
@@ -19,7 +19,6 @@ package org.apache.calcite.sql.type;
 import org.apache.calcite.rel.type.RelDataTypeSystem;
 import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.util.SerializableCharset;
-import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 
@@ -104,7 +103,7 @@ public class BasicSqlType extends AbstractSqlType {
     try {
       ret = (BasicSqlType) this.clone();
     } catch (CloneNotSupportedException e) {
-      throw Util.newInternal(e);
+      throw new AssertionError(e);
     }
     ret.isNullable = nullable;
     ret.computeDigest();
@@ -124,7 +123,7 @@ public class BasicSqlType extends AbstractSqlType {
     try {
       ret = (BasicSqlType) this.clone();
     } catch (CloneNotSupportedException e) {
-      throw Util.newInternal(e);
+      throw new AssertionError(e);
     }
     ret.wrappedCharset = SerializableCharset.forCharset(charset);
     ret.collation = collation;

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
index f0c5564..e1c028f 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/ReturnTypes.java
@@ -28,7 +28,6 @@ import org.apache.calcite.sql.SqlCollation;
 import org.apache.calcite.sql.SqlOperatorBinding;
 import org.apache.calcite.sql.SqlUtil;
 import org.apache.calcite.util.Glossary;
-import org.apache.calcite.util.Util;
 
 import com.google.common.base.Preconditions;
 
@@ -627,14 +626,12 @@ public abstract class ReturnTypes {
               .createSqlType(typeName, typePrecision);
           if (null != pickedCollation) {
             RelDataType pickedType;
-            if (argType0.getCollation().equals(
-                pickedCollation)) {
+            if (argType0.getCollation().equals(pickedCollation)) {
               pickedType = argType0;
-            } else if (argType1.getCollation().equals(
-                pickedCollation)) {
+            } else if (argType1.getCollation().equals(pickedCollation)) {
               pickedType = argType1;
             } else {
-              throw Util.newInternal("should never come here");
+              throw new AssertionError("should never come here");
             }
             ret =
                 opBinding.getTypeFactory()

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
index efe2bb7..27db0a9 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeAssignmentRules.java
@@ -16,8 +16,6 @@
  */
 package org.apache.calcite.sql.type;
 
-import org.apache.calcite.util.Util;
-
 import java.util.EnumSet;
 import java.util.HashMap;
 import java.util.HashSet;
@@ -318,12 +316,11 @@ public class SqlTypeAssignmentRules {
       return true;
     }
 
-    Set<SqlTypeName> rule = ruleset.get(to);
-    if (null == rule) {
+    final Set<SqlTypeName> rule = ruleset.get(to);
+    if (rule == null) {
       // if you hit this assert, see the constructor of this class on how
       // to add new rule
-      throw Util.newInternal(
-          "No assign rules for " + to + " defined");
+      throw new AssertionError("No assign rules for " + to + " defined");
     }
 
     return rule.contains(from);

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
index ee7b41a..e2162e1 100644
--- a/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
+++ b/core/src/main/java/org/apache/calcite/sql/type/SqlTypeUtil.java
@@ -80,15 +80,15 @@ public abstract class SqlTypeUtil {
       }
 
       if (t0.getCharset() == null) {
-        throw Util.newInternal("RelDataType object should have been assigned a "
-            + "(default) charset when calling deriveType");
+        throw new AssertionError("RelDataType object should have been assigned "
+            + "a (default) charset when calling deriveType");
       } else if (!t0.getCharset().equals(t1.getCharset())) {
         return false;
       }
 
       if (t0.getCollation() == null) {
-        throw Util.newInternal("RelDataType object should have been assigned a "
-            + "(default) collation when calling deriveType");
+        throw new AssertionError("RelDataType object should have been assigned "
+            + "a (default) collation when calling deriveType");
       } else if (!t0.getCollation().getCharset().equals(
           t1.getCollation().getCharset())) {
         return false;
@@ -577,7 +577,7 @@ public abstract class SqlTypeUtil {
     case DECIMAL:
       return NumberUtil.getMinUnscaled(type.getPrecision()).longValue();
     default:
-      throw Util.newInternal("getMinValue(" + typeName + ")");
+      throw new AssertionError("getMinValue(" + typeName + ")");
     }
   }
 
@@ -599,7 +599,7 @@ public abstract class SqlTypeUtil {
     case DECIMAL:
       return NumberUtil.getMaxUnscaled(type.getPrecision()).longValue();
     default:
-      throw Util.newInternal("getMaxValue(" + typeName + ")");
+      throw new AssertionError("getMaxValue(" + typeName + ")");
     }
   }
 

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/util/ReflectiveSqlOperatorTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/util/ReflectiveSqlOperatorTable.java b/core/src/main/java/org/apache/calcite/sql/util/ReflectiveSqlOperatorTable.java
index 494e600..a9d2ae1 100644
--- a/core/src/main/java/org/apache/calcite/sql/util/ReflectiveSqlOperatorTable.java
+++ b/core/src/main/java/org/apache/calcite/sql/util/ReflectiveSqlOperatorTable.java
@@ -25,7 +25,6 @@ import org.apache.calcite.sql.SqlSyntax;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
-import com.google.common.base.Throwables;
 import com.google.common.collect.HashMultimap;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.Multimap;
@@ -72,7 +71,8 @@ public abstract class ReflectiveSqlOperatorTable implements SqlOperatorTable {
           register(op);
         }
       } catch (IllegalArgumentException | IllegalAccessException e) {
-        throw Throwables.propagate(e);
+        Util.throwIfUnchecked(e.getCause());
+        throw new RuntimeException(e.getCause());
       }
     }
   }
@@ -135,12 +135,6 @@ public abstract class ReflectiveSqlOperatorTable implements SqlOperatorTable {
    */
   public void register(SqlOperator op) {
     operators.put(new Key(op.getName(), op.getSyntax()), op);
-    if (op instanceof SqlFunction) {
-      SqlFunction function = (SqlFunction) op;
-      SqlFunctionCategory funcType = function.getFunctionType();
-      assert funcType != null
-          : "Function type for " + function.getName() + " not set";
-    }
   }
 
   public List<SqlOperator> getOperatorList() {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
index b6bd058..d0caa7c 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/AbstractNamespace.java
@@ -22,6 +22,7 @@ import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.util.Pair;
 import org.apache.calcite.util.Util;
 
+import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableList;
 
 import java.util.List;
@@ -78,12 +79,10 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
     case UNVALIDATED:
       try {
         status = SqlValidatorImpl.Status.IN_PROGRESS;
-        Util.permAssert(
-            rowType == null,
+        Preconditions.checkArgument(rowType == null,
             "Namespace.rowType must be null before validate has been called");
         RelDataType type = validateImpl(targetRowType);
-        Util.permAssert(
-            type != null,
+        Preconditions.checkArgument(type != null,
             "validateImpl() returned null");
         setType(type);
       } finally {
@@ -91,7 +90,7 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
       }
       break;
     case IN_PROGRESS:
-      throw Util.newInternal("todo: Cycle detected during type-checking");
+      throw new AssertionError("Cycle detected during type-checking");
     case VALID:
       break;
     default:
@@ -114,7 +113,7 @@ abstract class AbstractNamespace implements SqlValidatorNamespace {
   public RelDataType getRowType() {
     if (rowType == null) {
       validator.validateNamespace(this, validator.unknownType);
-      Util.permAssert(rowType != null, "validate must set rowType");
+      Preconditions.checkArgument(rowType != null, "validate must set rowType");
     }
     return rowType;
   }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/validate/SetopNamespace.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SetopNamespace.java b/core/src/main/java/org/apache/calcite/sql/validate/SetopNamespace.java
index adc6e6d..d6f65e1 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SetopNamespace.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SetopNamespace.java
@@ -20,7 +20,6 @@ import org.apache.calcite.rel.type.RelDataType;
 import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlKind;
 import org.apache.calcite.sql.SqlNode;
-import org.apache.calcite.util.Util;
 
 import static org.apache.calcite.util.Static.RESOURCE;
 
@@ -107,7 +106,7 @@ public class SetopNamespace extends AbstractNamespace {
           scope,
           call);
     default:
-      throw Util.newInternal("Not a query: " + call.getKind());
+      throw new AssertionError("Not a query: " + call.getKind());
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/validate/SqlIdentifierMoniker.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlIdentifierMoniker.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlIdentifierMoniker.java
index e4f498b..29a75e0 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlIdentifierMoniker.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlIdentifierMoniker.java
@@ -17,7 +17,8 @@
 package org.apache.calcite.sql.validate;
 
 import org.apache.calcite.sql.SqlIdentifier;
-import org.apache.calcite.util.Util;
+
+import com.google.common.base.Preconditions;
 
 import java.util.List;
 
@@ -36,8 +37,7 @@ public class SqlIdentifierMoniker implements SqlMoniker {
    * Creates an SqlIdentifierMoniker.
    */
   public SqlIdentifierMoniker(SqlIdentifier id) {
-    Util.pre(id != null, "id != null");
-    this.id = id;
+    this.id = Preconditions.checkNotNull(id);
   }
 
   //~ Methods ----------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
index 3766258..f0d0ce9 100644
--- a/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
+++ b/core/src/main/java/org/apache/calcite/sql/validate/SqlValidatorImpl.java
@@ -597,7 +597,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     }
     final SqlValidatorNamespace ns = getNamespace(outermostNode);
     if (ns == null) {
-      throw Util.newInternal("Not a query: " + outermostNode);
+      throw new AssertionError("Not a query: " + outermostNode);
     }
     Collection<SqlMoniker> hintList = Sets.newTreeSet(SqlMoniker.COMPARATOR);
     lookupSelectHints(ns, pos, hintList);
@@ -1492,8 +1492,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   void setValidatedNodeTypeImpl(SqlNode node, RelDataType type) {
-    Util.pre(type != null, "type != null");
-    Util.pre(node != null, "node != null");
+    Preconditions.checkNotNull(type);
+    Preconditions.checkNotNull(node);
     if (type.equals(unknownType)) {
       // don't set anything until we know what it is, and don't overwrite
       // a known type with the unknown type
@@ -1505,8 +1505,8 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   public RelDataType deriveType(
       SqlValidatorScope scope,
       SqlNode expr) {
-    Util.pre(scope != null, "scope != null");
-    Util.pre(expr != null, "expr != null");
+    Preconditions.checkNotNull(scope);
+    Preconditions.checkNotNull(expr);
 
     // if we already know the type, no need to re-derive
     RelDataType type = nodeToTypeMap.get(expr);
@@ -1518,7 +1518,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       return ns.getType();
     }
     type = deriveTypeImpl(scope, expr);
-    Util.permAssert(
+    Preconditions.checkArgument(
         type != null,
         "SqlValidator.deriveTypeInternal returned null");
     setValidatedNodeTypeImpl(expr, type);
@@ -2842,7 +2842,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlNode node,
       RelDataType targetRowType,
       SqlValidatorScope scope) {
-    Util.pre(targetRowType != null, "targetRowType != null");
+    Preconditions.checkNotNull(targetRowType);
     switch (node.getKind()) {
     case AS:
       validateFrom(
@@ -2870,7 +2870,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
   }
 
   protected void validateOver(SqlCall call, SqlValidatorScope scope) {
-    throw Util.newInternal("OVER unexpected in this context");
+    throw new AssertionError("OVER unexpected in this context");
   }
 
   protected void validateJoin(SqlJoin join, SqlValidatorScope scope) {
@@ -2887,10 +2887,10 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
     // Validate condition.
     switch (conditionType) {
     case NONE:
-      Util.permAssert(condition == null, "condition == null");
+      Preconditions.checkArgument(condition == null);
       break;
     case ON:
-      Util.permAssert(condition != null, "condition != null");
+      Preconditions.checkArgument(condition != null);
       SqlNode expandedCondition = expand(condition, joinScope);
       join.setOperand(5, expandedCondition);
       condition = join.getCondition();
@@ -2900,7 +2900,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       SqlNodeList list = (SqlNodeList) condition;
 
       // Parser ensures that using clause is not empty.
-      Util.permAssert(list.size() > 0, "Empty USING clause");
+      Preconditions.checkArgument(list.size() > 0, "Empty USING clause");
       for (int i = 0; i < list.size(); i++) {
         SqlIdentifier id = (SqlIdentifier) list.get(i);
         final RelDataType leftColType = validateUsingCol(id, left);
@@ -3375,8 +3375,7 @@ public class SqlValidatorImpl implements SqlValidatorWithHints {
       }
     }
     final SqlValidatorScope orderScope = getOrderScope(select);
-
-    Util.permAssert(orderScope != null, "orderScope != null");
+    Preconditions.checkNotNull(orderScope != null);
 
     List<SqlNode> expandList = new ArrayList<>();
     for (SqlNode orderItem : orderList) {

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
index fcd3f4c..4d87df1 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/ReflectiveConvertletTable.java
@@ -21,7 +21,8 @@ import org.apache.calcite.sql.SqlCall;
 import org.apache.calcite.sql.SqlNode;
 import org.apache.calcite.sql.SqlOperator;
 import org.apache.calcite.sql.parser.SqlParserPos;
-import org.apache.calcite.util.Util;
+
+import com.google.common.base.Preconditions;
 
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Method;
@@ -38,7 +39,7 @@ import java.util.Map;
 public class ReflectiveConvertletTable implements SqlRexConvertletTable {
   //~ Instance fields --------------------------------------------------------
 
-  private final Map<Object, Object> map = new HashMap<Object, Object>();
+  private final Map<Object, Object> map = new HashMap<>();
 
   //~ Constructors -----------------------------------------------------------
 
@@ -77,25 +78,16 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
     if (!SqlNode.class.isAssignableFrom(parameterType)) {
       return;
     }
-    map.put(
-        parameterType,
+    map.put(parameterType,
         new SqlRexConvertlet() {
           public RexNode convertCall(
               SqlRexContext cx,
               SqlCall call) {
             try {
-              return (RexNode) method.invoke(
-                  ReflectiveConvertletTable.this,
-                  cx,
-                  call);
-            } catch (IllegalAccessException e) {
-              throw Util.newInternal(
-                  e,
-                  "while converting " + call);
-            } catch (InvocationTargetException e) {
-              throw Util.newInternal(
-                  e,
-                  "while converting " + call);
+              return (RexNode) method.invoke(ReflectiveConvertletTable.this,
+                  cx, call);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+              throw new RuntimeException("while converting " + call, e);
             }
           }
         });
@@ -132,26 +124,14 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
     if (!SqlCall.class.isAssignableFrom(parameterType)) {
       return;
     }
-    map.put(
-        opClass,
+    map.put(opClass,
         new SqlRexConvertlet() {
-          public RexNode convertCall(
-              SqlRexContext cx,
-              SqlCall call) {
+          public RexNode convertCall(SqlRexContext cx, SqlCall call) {
             try {
-              return (RexNode) method.invoke(
-                  ReflectiveConvertletTable.this,
-                  cx,
-                  call.getOperator(),
-                  call);
-            } catch (IllegalAccessException e) {
-              throw Util.newInternal(
-                  e,
-                  "while converting " + call);
-            } catch (InvocationTargetException e) {
-              throw Util.newInternal(
-                  e,
-                  "while converting " + call);
+              return (RexNode) method.invoke(ReflectiveConvertletTable.this,
+                  cx, call.getOperator(), call);
+            } catch (IllegalAccessException | InvocationTargetException e) {
+              throw new RuntimeException("while converting " + call, e);
             }
           }
         });
@@ -170,7 +150,7 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
 
     // Is there a convertlet for this class of operator
     // (e.g. SqlBinaryOperator)?
-    Class<? extends Object> clazz = op.getClass();
+    Class<?> clazz = op.getClass();
     while (clazz != null) {
       convertlet = (SqlRexConvertlet) map.get(clazz);
       if (convertlet != null) {
@@ -216,13 +196,10 @@ public class ReflectiveConvertletTable implements SqlRexConvertletTable {
           public RexNode convertCall(
               SqlRexContext cx,
               SqlCall call) {
-            Util.permAssert(
-                call.getOperator() == alias,
+            Preconditions.checkArgument(call.getOperator() == alias,
                 "call to wrong operator");
             final SqlCall newCall =
-                target.createCall(
-                    SqlParserPos.ZERO,
-                    call.getOperandList());
+                target.createCall(SqlParserPos.ZERO, call.getOperandList());
             return cx.convertExpression(newCall);
           }
         });

http://git-wip-us.apache.org/repos/asf/calcite/blob/603648bc/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
index 723fb78..853e69b 100644
--- a/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
+++ b/core/src/main/java/org/apache/calcite/sql2rel/RelFieldTrimmer.java
@@ -909,7 +909,7 @@ public class RelFieldTrimmer implements ReflectiveVisitor {
     if (!inputMapping.isIdentity()) {
       // We asked for all fields. Can't believe that the child decided
       // to permute them!
-      throw Util.newInternal(
+      throw new AssertionError(
           "Expected identity mapping, got " + inputMapping);
     }